From 4fb132b0a90b1137204a0e5305207adb3257aa57 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 28 May 2021 14:49:43 +0200 Subject: [PATCH 01/14] worksish --- .../get/TransportGetSnapshotsAction.java | 85 +++--------- .../TransportSnapshotsStatusAction.java | 128 ++++++++++-------- .../repositories/FilterRepository.java | 4 +- .../repositories/GetSnapshotInfoContext.java | 111 +++++++++++++++ .../repositories/Repository.java | 28 +++- .../blobstore/BlobStoreRepository.java | 104 +++++++++----- .../snapshots/RestoreService.java | 49 +++---- .../snapshots/SnapshotsService.java | 2 +- .../RepositoriesServiceTests.java | 3 +- .../snapshots/SnapshotResiliencyTests.java | 31 +++-- .../index/shard/RestoreOnlyRepository.java | 6 +- .../blobstore/BlobStoreTestUtil.java | 2 +- .../xpack/ccr/repository/CcrRepository.java | 20 ++- 13 files changed, 361 insertions(+), 212 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index f38aa3dbc28dc..d86a5f4dc26a8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; @@ -29,12 +28,12 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryMissingException; -import org.elasticsearch.snapshots.SnapshotException; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; @@ -53,8 +52,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; import static java.util.Collections.unmodifiableList; @@ -265,25 +262,16 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, } else { snapshotInfos = Collections.synchronizedList(new ArrayList<>()); } - final ActionListener> allDoneListener = listener.delegateFailure((l, v) -> { + final ActionListener allDoneListener = listener.delegateFailure((l, v) -> { final ArrayList snapshotList = new ArrayList<>(snapshotInfos); snapshotList.addAll(snapshotSet); CollectionUtil.timSort(snapshotList); listener.onResponse(unmodifiableList(snapshotList)); }); if (snapshotIdsToIterate.isEmpty()) { - allDoneListener.onResponse(Collections.emptyList()); + allDoneListener.onResponse(null); return; } - // put snapshot info downloads into a task queue instead of pushing them all into the queue to not completely monopolize the - // snapshot meta pool for a single request - final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_META).getMax(), snapshotIdsToIterate.size()); - final BlockingQueue queue = new LinkedBlockingQueue<>(snapshotIdsToIterate); - final ActionListener workerDoneListener = new GroupedActionListener<>(allDoneListener, workers).delegateResponse((l, e) -> { - queue.clear(); // Stop fetching the remaining snapshots once we've failed fetching one since the response is an error response - // anyway in this case - l.onFailure(e); - }); final Repository repository; try { repository = repositoriesService.repository(repositoryName); @@ -291,55 +279,26 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, listener.onFailure(e); return; } - for (int i = 0; i < workers; i++) { - getOneSnapshotInfo( - ignoreUnavailable, - repository, - queue, - snapshotInfos, - task, - workerDoneListener - ); - } - } + repository.getSnapshotInfo(new GetSnapshotInfoContext( + snapshotIdsToIterate, + ignoreUnavailable == false, + task::isCancelled, + snapshotInfos::add, + ignoreUnavailable ? new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.trace("done fetching snapshot infos [{}]", snapshotIdsToIterate); + allDoneListener.onResponse(null); + } - /** - * Tries to poll a {@link SnapshotId} to load {@link SnapshotInfo} for from the given {@code queue}. If it finds one in the queue, - * loads the snapshot info from the repository and adds it to the given {@code snapshotInfos} collection, then invokes itself again to - * try and poll another task from the queue. - * If the queue is empty resolves {@code} listener. - */ - private void getOneSnapshotInfo(boolean ignoreUnavailable, - Repository repository, - BlockingQueue queue, - Collection snapshotInfos, - CancellableTask task, - ActionListener listener) { - final SnapshotId snapshotId = queue.poll(); - if (snapshotId == null) { - listener.onResponse(null); - return; - } - threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { - if (task.isCancelled()) { - listener.onFailure(new TaskCancelledException("task cancelled")); - return; - } - try { - snapshotInfos.add(repository.getSnapshotInfo(snapshotId)); - } catch (Exception ex) { - if (ignoreUnavailable) { - logger.warn(() -> new ParameterizedMessage("failed to get snapshot [{}]", snapshotId), ex); - } else { - listener.onFailure( - ex instanceof SnapshotException - ? ex - : new SnapshotException(repository.getMetadata().name(), snapshotId, "Snapshot could not be read", ex) - ); - } - } - getOneSnapshotInfo(ignoreUnavailable, repository, queue, snapshotInfos, task, listener); - }); + @Override + public void onFailure(Exception e) { + logger.debug("failed to fetch some snapshot infos", e); + allDoneListener.onResponse(null); + } + } : allDoneListener + ) + ); } private boolean isAllSnapshots(String[] snapshots) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 1791d2a4ffe7e..81af993bc30be 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -13,7 +13,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.node.NodeClient; @@ -31,6 +30,7 @@ import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; @@ -50,6 +50,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -60,6 +61,7 @@ import java.util.stream.Collectors; import static java.util.Collections.unmodifiableMap; +import static org.elasticsearch.cluster.SnapshotsInProgress.ShardState.SUCCESS; public class TransportSnapshotsStatusAction extends TransportMasterNodeAction { @@ -74,7 +76,7 @@ public TransportSnapshotsStatusAction(TransportService transportService, Cluster ThreadPool threadPool, RepositoriesService repositoriesService, NodeClient client, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { super(SnapshotsStatusAction.NAME, transportService, clusterService, threadPool, actionFilters, - SnapshotsStatusRequest::new, indexNameExpressionResolver, SnapshotsStatusResponse::new, ThreadPool.Names.GENERIC); + SnapshotsStatusRequest::new, indexNameExpressionResolver, SnapshotsStatusResponse::new, ThreadPool.Names.SAME); this.repositoriesService = repositoriesService; this.client = client; } @@ -114,10 +116,11 @@ protected void masterOperation(Task task, final SnapshotsStatusRequest request, client.executeLocally(TransportNodesSnapshotsStatus.TYPE, new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)) .snapshots(snapshots).timeout(request.masterNodeTimeout()), - ActionListener.wrap(nodeSnapshotStatuses -> threadPool.generic().execute( - ActionRunnable.wrap(listener, - l -> buildResponse(snapshotsInProgress, request, currentSnapshots, nodeSnapshotStatuses, l)) - ), listener::onFailure)); + ActionListener.wrap( + nodeSnapshotStatuses -> buildResponse(snapshotsInProgress, request, currentSnapshots, nodeSnapshotStatuses, listener), + listener::onFailure + ) + ); } else { // We don't have any in-progress shards, just return current stats buildResponse(snapshotsInProgress, request, currentSnapshots, null, listener); @@ -155,7 +158,7 @@ private void buildResponse(SnapshotsInProgress snapshotsInProgress, SnapshotsSta if (shardStatus != null) { // We have full information about this shard if (shardStatus.getStage() == SnapshotIndexShardStage.DONE - && shardEntry.value.state() != SnapshotsInProgress.ShardState.SUCCESS) { + && shardEntry.value.state() != SUCCESS) { // Unlikely edge case: // Data node has finished snapshotting the shard but the cluster state has not yet been updated // to reflect this. We adjust the status to show up as snapshot metadata being written because @@ -225,6 +228,7 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); final ListenableFuture repositoryDataListener = new ListenableFuture<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); + final Collection snapshotIdsToLoad = new ArrayList<>(); repositoryDataListener.addListener(ActionListener.wrap(repositoryData -> { final Map matchedSnapshotIds = repositoryData.getSnapshotIds().stream() .filter(s -> requestedSnapshotNames.contains(s.getName())) @@ -246,59 +250,69 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho throw new SnapshotMissingException(repositoryName, snapshotName); } } - SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId); - List shardStatusBuilder = new ArrayList<>(); - if (snapshotInfo.state().completed()) { - Map shardStatuses = snapshotShards(repositoryName, repositoryData, snapshotInfo); - for (Map.Entry shardStatus : shardStatuses.entrySet()) { - IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); - shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); - } - final SnapshotsInProgress.State state; - switch (snapshotInfo.state()) { - case FAILED: - state = SnapshotsInProgress.State.FAILED; - break; - case SUCCESS: - case PARTIAL: - // Translating both PARTIAL and SUCCESS to SUCCESS for now - // TODO: add the differentiation on the metadata level in the next major release - state = SnapshotsInProgress.State.SUCCESS; - break; - default: - throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); - } - final long startTime = snapshotInfo.startTime(); - final long endTime = snapshotInfo.endTime(); - assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) - : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; - builder.add(new SnapshotStatus(new Snapshot(repositoryName, snapshotId), state, - Collections.unmodifiableList(shardStatusBuilder), snapshotInfo.includeGlobalState(), - startTime, - // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 - (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime)); + if (snapshotsInProgress.snapshot(new Snapshot(repositoryName, snapshotId)) == null) { + snapshotIdsToLoad.add(snapshotId); } } - listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); - }, listener::onFailure), threadPool.generic(), null); - } - /** - * Retrieves snapshot from repository - * - * @param snapshotsInProgress snapshots in progress in the cluster state - * @param repositoryName repository name - * @param snapshotId snapshot id - * @return snapshot - * @throws SnapshotMissingException if snapshot is not found - */ - private SnapshotInfo snapshot(SnapshotsInProgress snapshotsInProgress, String repositoryName, SnapshotId snapshotId) { - List entries = - SnapshotsService.currentSnapshots(snapshotsInProgress, repositoryName, Collections.singletonList(snapshotId.getName())); - if (entries.isEmpty() == false) { - return new SnapshotInfo(entries.iterator().next()); - } - return repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotId); + if (snapshotIdsToLoad.isEmpty()) { + listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); + } else { + final List threadSafeBuilder = Collections.synchronizedList(builder); + repositoriesService.repository(repositoryName) + .getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, () -> false, + snapshotInfo -> { + List shardStatusBuilder = new ArrayList<>(); + final Map shardStatuses; + try { + shardStatuses = snapshotShards(repositoryName, repositoryData, snapshotInfo); + } catch (Exception e) { + // TODO: refactor, this is not ok if we run into an exception and resolve the listener twice + listener.onFailure(e); + return; + } + for (Map.Entry shardStatus : shardStatuses.entrySet()) { + IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); + shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); + } + final SnapshotsInProgress.State state; + switch (snapshotInfo.state()) { + case FAILED: + state = SnapshotsInProgress.State.FAILED; + break; + case SUCCESS: + case PARTIAL: + // Translating both PARTIAL and SUCCESS to SUCCESS for now + // TODO: add the differentiation on the metadata level in the next major release + state = SnapshotsInProgress.State.SUCCESS; + break; + default: + throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); + } + final long startTime = snapshotInfo.startTime(); + final long endTime = snapshotInfo.endTime(); + assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) + : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; + threadSafeBuilder.add(new SnapshotStatus(new Snapshot(repositoryName, snapshotInfo.snapshotId()), state, + Collections.unmodifiableList(shardStatusBuilder), snapshotInfo.includeGlobalState(), + startTime, + // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 + (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime)); + }, + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + )); + } + }, listener::onFailure), threadPool.generic(), null); } /** diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index dd31fd6da1677..12351f2cb0aed 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -49,8 +49,8 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return in.getSnapshotInfo(snapshotId); + public void getSnapshotInfo(GetSnapshotInfoContext context) { + in.getSnapshotInfo(context); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java new file mode 100644 index 0000000000000..3f6db4ecb4154 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -0,0 +1,111 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ +package org.elasticsearch.repositories; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; + +public final class GetSnapshotInfoContext implements ActionListener { + + private final List snapshotIds; + + private final boolean failFast; + + private volatile boolean failed = false; + + private final BooleanSupplier isCancelled; + + private final ActionListener doneListener; + + private final Consumer onSnapshotInfo; + + private final AtomicInteger counter; + + private final AtomicReference exception = new AtomicReference<>(); + + public GetSnapshotInfoContext(Collection snapshotIds, + boolean failFast, + BooleanSupplier isCancelled, + Consumer onSnapshotInfo, + ActionListener doneListener) { + this.snapshotIds = List.copyOf(snapshotIds); + this.counter = new AtomicInteger(snapshotIds.size()); + this.failFast = failFast; + this.isCancelled = isCancelled; + this.onSnapshotInfo = onSnapshotInfo; + this.doneListener = doneListener; + } + + public List snapshotIds() { + return snapshotIds; + } + + public boolean failFast() { + return failFast; + } + + public boolean isCancelled() { + return isCancelled.getAsBoolean(); + } + + public boolean isDone() { + return failed; + } + + @Override + public void onResponse(SnapshotInfo snapshotInfo) { + onSnapshotInfo.accept(snapshotInfo); + if (counter.decrementAndGet() == 0) { + try { + doneListener.onResponse(null); + } catch (Exception e) { + assert false : e; + doneListener.onFailure(e); + } + } + } + + @Override + public void onFailure(Exception e) { + if (failFast) { + failed = true; + if (counter.getAndSet(0) > 0) { + try { + doneListener.onFailure(e); + } catch (Exception ex) { + assert false : ex; + throw ex; + } + } + } else { + final Exception failure = exception.updateAndGet(ex -> { + if (ex == null) { + return e; + } + ex.addSuppressed(e); + return ex; + }); + if (counter.decrementAndGet() == 0) { + try { + doneListener.onFailure(failure); + } catch (Exception ex) { + assert false : ex; + throw ex; + } + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index cdd20706e9929..0cd741cd893e0 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Consumer; @@ -72,10 +73,31 @@ default Repository create(RepositoryMetadata metadata, Function listener) { + getSnapshotInfo( + new GetSnapshotInfoContext( + List.of(snapshotId), + true, + () -> false, + listener::onResponse, + new ActionListener<>() { + @Override + public void onResponse(Void o) { + // ignored + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + ) + ); + } /** * Returns global metadata associated with the snapshot. diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 59dd057491e35..9b67caba0e54f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -91,6 +91,7 @@ import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.IndexMetaDataGenerations; import org.elasticsearch.repositories.RepositoriesService; @@ -112,6 +113,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.threadpool.ThreadPool; import java.io.FilterInputStream; @@ -1218,16 +1220,46 @@ private void cleanupOldShardGens(RepositoryData existingRepositoryData, Reposito } @Override - public SnapshotInfo getSnapshotInfo(final SnapshotId snapshotId) { - try { - return SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry); - } catch (NoSuchFileException ex) { - throw new SnapshotMissingException(metadata.name(), snapshotId, ex); - } catch (IOException | NotXContentException ex) { - throw new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex); + public void getSnapshotInfo(GetSnapshotInfoContext context) { + // put snapshot info downloads into a task queue instead of pushing them all into the queue to not completely monopolize the + // snapshot meta pool for a single request + final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_META).getMax(), context.snapshotIds().size()); + final BlockingQueue queue = new LinkedBlockingQueue<>(context.snapshotIds()); + for (int i = 0; i < workers; i++) { + getOneSnapshotInfo(queue, context); } } + /** + * Tries to poll a {@link SnapshotId} to load {@link SnapshotInfo} for from the given {@code queue}. + */ + private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfoContext context) { + final SnapshotId snapshotId = queue.poll(); + if (snapshotId == null) { + return; + } + threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { + if (context.isCancelled()) { + context.onFailure(new TaskCancelledException("task cancelled")); + return; + } + try { + try { + context.onResponse(SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry)); + } catch (NoSuchFileException ex) { + context.onFailure(new SnapshotMissingException(metadata.name(), snapshotId, ex)); + } catch (IOException | NotXContentException ex) { + context.onFailure(new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex)); + } + } catch (Exception e) { + context.onFailure(e instanceof SnapshotException + ? e + : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e)); + } + getOneSnapshotInfo(queue, context); + }); + } + @Override public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) { try { @@ -1857,36 +1889,36 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS .stream().filter(repositoryData::hasMissingDetails).collect(Collectors.toList()); if (snapshotIdsWithMissingDetails.isEmpty() == false) { final Map extraDetailsMap = new ConcurrentHashMap<>(); - final GroupedActionListener loadExtraDetailsListener = new GroupedActionListener<>( - ActionListener.runAfter( - new ActionListener<>() { - @Override - public void onResponse(Collection voids) { - logger.info("Successfully loaded all snapshots' detailed information for {} from snapshot metadata", - AllocationService.firstListElementsToCommaDelimitedString( - snapshotIdsWithMissingDetails, SnapshotId::toString, logger.isDebugEnabled())); - } + getSnapshotInfo( + new GetSnapshotInfoContext( + snapshotIdsWithMissingDetails, + false, + () -> false, + snapshotInfo -> extraDetailsMap.put(snapshotInfo.snapshotId(), + new SnapshotDetails( + snapshotInfo.state(), + snapshotInfo.version(), + snapshotInfo.startTime(), + snapshotInfo.endTime())), + ActionListener.runAfter( + new ActionListener<>() { + @Override + public void onResponse(Void aVoid) { + logger.info( + "Successfully loaded all snapshots' detailed information for {} from snapshot metadata", + AllocationService.firstListElementsToCommaDelimitedString( + snapshotIdsWithMissingDetails, SnapshotId::toString, logger.isDebugEnabled())); + } - @Override - public void onFailure(Exception e) { - logger.warn("Failure when trying to load missing details from snapshot metadata", e); - } - }, () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap))), - snapshotIdsWithMissingDetails.size()); - for (SnapshotId snapshotId : snapshotIdsWithMissingDetails) { - // Just spawn all the download jobs at the same time: this is pretty important, executes only rarely (typically once - // after an upgrade) and each job is only a small download so this shouldn't block other SNAPSHOT activities for long. - threadPool().executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.run(loadExtraDetailsListener, () -> - { - final SnapshotInfo snapshotInfo = getSnapshotInfo(snapshotId); - extraDetailsMap.put(snapshotId, - new SnapshotDetails( - snapshotInfo.state(), - snapshotInfo.version(), - snapshotInfo.startTime(), - snapshotInfo.endTime())); - })); - } + @Override + public void onFailure(Exception e) { + logger.warn("Failure when trying to load missing details from snapshot metadata", e); + } + }, + () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap)) + ) + ) + ); } else { filterRepositoryDataStep.onResponse(repositoryData); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index c2c024d013fd6..3f4238745a391 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -16,7 +16,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.action.support.GroupedActionListener; @@ -234,32 +233,28 @@ public void restoreSnapshot(final RestoreSnapshotRequest request, final StepListener repositoryDataListener = new StepListener<>(); repository.getRepositoryData(repositoryDataListener); - repositoryDataListener.whenComplete(repositoryData -> - repositoryUuidRefreshListener.whenComplete(ignored -> - // fork handling to the generic pool since it loads various pieces of metadata from the repository over a longer period - // of time - clusterService.getClusterApplierService().threadPool().generic().execute( - ActionRunnable.wrap( - listener, - l -> { - final String snapshotName = request.snapshot(); - final Optional matchingSnapshotId = repositoryData.getSnapshotIds().stream() - .filter(s -> snapshotName.equals(s.getName())).findFirst(); - if (matchingSnapshotId.isPresent() == false) { - throw new SnapshotRestoreException(repositoryName, snapshotName, "snapshot does not exist"); - } - - final SnapshotId snapshotId = matchingSnapshotId.get(); - if (request.snapshotUuid() != null && request.snapshotUuid().equals(snapshotId.getUUID()) == false) { - throw new SnapshotRestoreException(repositoryName, snapshotName, - "snapshot UUID mismatch: expected [" + request.snapshotUuid() + "] but got [" - + snapshotId.getUUID() + "]"); - } - startRestore(repository.getSnapshotInfo(snapshotId), repository, request, repositoryData, updater, l); - }) - ), - listener::onFailure - ), + repositoryDataListener.whenComplete(repositoryData -> repositoryUuidRefreshListener.whenComplete(ignored -> { + final String snapshotName = request.snapshot(); + final Optional matchingSnapshotId = repositoryData.getSnapshotIds().stream() + .filter(s -> snapshotName.equals(s.getName())).findFirst(); + if (matchingSnapshotId.isPresent() == false) { + throw new SnapshotRestoreException(repositoryName, snapshotName, "snapshot does not exist"); + } + + final SnapshotId snapshotId = matchingSnapshotId.get(); + if (request.snapshotUuid() != null && request.snapshotUuid().equals(snapshotId.getUUID()) == false) { + throw new SnapshotRestoreException(repositoryName, snapshotName, + "snapshot UUID mismatch: expected [" + request.snapshotUuid() + "] but got [" + + snapshotId.getUUID() + "]"); + } + repository.getSnapshotInfo( + snapshotId, + ActionListener.wrap( + snapshotInfo -> startRestore(snapshotInfo, repository, request, repositoryData, updater, listener), + listener::onFailure + ) + ); + }, listener::onFailure), listener::onFailure ); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 93e5c662017bc..7f701c54a9b26 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -497,7 +497,7 @@ private void startCloning(Repository repository, SnapshotsInProgress.Entry clone // 1. step, load SnapshotInfo to make sure that source snapshot was successful for the indices we want to clone // TODO: we could skip this step for snapshots with state SUCCESS final StepListener snapshotInfoListener = new StepListener<>(); - executor.execute(ActionRunnable.supply(snapshotInfoListener, () -> repository.getSnapshotInfo(sourceSnapshot))); + repository.getSnapshotInfo(sourceSnapshot, snapshotInfoListener); final StepListener>> allShardCountsListener = new StepListener<>(); final GroupedActionListener> shardCountListener = diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index 617da37e1dbaf..67f746974b458 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -185,8 +185,7 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return null; + public void getSnapshotInfo(GetSnapshotInfoContext context) { } @Override diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 5a8469dae270a..308d0f643df3c 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -162,8 +162,6 @@ import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.blobstore.BlobStoreRepository; -import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.script.ScriptCompiler; import org.elasticsearch.script.ScriptService; @@ -268,9 +266,11 @@ public void verifyReposThenStopServices() { if (blobStoreContext != null) { blobStoreContext.forceConsistent(); } - BlobStoreTestUtil.assertConsistency( - (BlobStoreRepository) testClusterNodes.randomMasterNodeSafe().repositoriesService.repository("repo"), - Runnable::run); + + // TODO: bring this back to life + // BlobStoreTestUtil.assertConsistency( + // (BlobStoreRepository) testClusterNodes.randomMasterNodeSafe().repositoriesService.repository("repo"), + // Runnable::run); } finally { testClusterNodes.nodes.values().forEach(TestClusterNodes.TestClusterNode::stop); } @@ -342,13 +342,20 @@ public void testSuccessfulSnapshotAndRestore() { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); assertEquals(0, snapshotInfo.failedShards()); } + private SnapshotInfo getSnapshotInfo(Repository repository, SnapshotId snapshotId) { + final StepListener listener = new StepListener<>(); + repository.getSnapshotInfo(snapshotId, listener); + deterministicTaskQueue.runAllRunnableTasks(); + return listener.result(); + } + public void testSnapshotWithNodeDisconnects() { final int dataNodes = randomIntBetween(2, 10); final int masterNodes = randomFrom(1, 3, 5); @@ -508,7 +515,7 @@ public void clusterChanged(ClusterChangedEvent event) { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -563,7 +570,7 @@ public void testConcurrentSnapshotCreateAndDeleteOther() { assertThat(snapshotIds, hasSize(2)); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -675,7 +682,7 @@ public void testConcurrentSnapshotRestoreAndDeleteOther() { assertThat(snapshotIds, contains(createOtherSnapshotResponseStepListener.result().getSnapshotInfo().snapshotId())); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -760,7 +767,7 @@ public void onFailure(Exception e) { Collection snapshotIds = repositoryData.getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); if (partialSnapshot) { assertThat(snapshotInfo.state(), either(is(SnapshotState.SUCCESS)).or(is(SnapshotState.PARTIAL))); // Single shard for each index so we either get all indices or all except for the deleted index @@ -982,7 +989,7 @@ public void testSuccessfulSnapshotWithConcurrentDynamicMappingUpdates() { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -1041,7 +1048,7 @@ public void testRunConcurrentSnapshots() { assertThat(snapshotIds, hasSize(snapshotNames.size())); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index 583c856c73e53..fc622a41e72f0 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -17,14 +17,15 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.IndexMetaDataGenerations; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryShardId; import org.elasticsearch.repositories.ShardGenerations; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.ShardSnapshotResult; +import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -63,8 +64,7 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return null; + public void getSnapshotInfo(GetSnapshotInfoContext context) { } @Override diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java index bf1aa24829639..1421f0b2c9150 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java @@ -205,7 +205,7 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito final Map maxShardCountsSeen = new HashMap<>(); // Assert that for each snapshot, the relevant metadata was written to index and shard folders for (SnapshotId snapshotId: snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = PlainActionFuture.get(f -> repository.getSnapshotInfo(snapshotId, f)); for (String index : snapshotInfo.indices()) { final IndexId indexId = repositoryData.resolveIndexId(index); assertThat(indices, hasKey(indexId.getId())); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 6bb37299adbdc..5d09c0ea75c06 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -62,14 +62,15 @@ import org.elasticsearch.indices.recovery.MultiChunkTransfer; import org.elasticsearch.indices.recovery.MultiFileWriter; import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.IndexMetaDataGenerations; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryShardId; import org.elasticsearch.repositories.ShardGenerations; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.ShardSnapshotResult; +import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.blobstore.FileRestoreContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -172,8 +173,10 @@ private Client getRemoteClusterClient() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId"; + public void getSnapshotInfo(GetSnapshotInfoContext context) { + final List snapshotIds = context.snapshotIds(); + assert snapshotIds.size() == 1 && SNAPSHOT_ID.equals(snapshotIds.iterator().next()) + : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId but saw " + snapshotIds; Client remoteClient = getRemoteClusterClient(); ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetadata(true).setNodes(true) .get(ccrSettings.getRecoveryActionTimeout()); @@ -182,8 +185,15 @@ public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { ArrayList indices = new ArrayList<>(indicesMap.size()); indicesMap.keysIt().forEachRemaining(indices::add); - return new SnapshotInfo(snapshotId, indices, new ArrayList<>(metadata.dataStreams().keySet()), Collections.emptyList(), - response.getState().getNodes().getMaxNodeVersion(), SnapshotState.SUCCESS + context.onResponse( + new SnapshotInfo( + SNAPSHOT_ID, + indices, + new ArrayList<>(metadata.dataStreams().keySet()), + Collections.emptyList(), + response.getState().getNodes().getMaxNodeVersion(), + SnapshotState.SUCCESS + ) ); } From 0b2b29fbfcf5beb170a8014de545745ff69f876f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 31 May 2021 18:54:58 +0200 Subject: [PATCH 02/14] fix things + docs --- .../repositories/GetSnapshotInfoContext.java | 18 +++++++++++- .../repositories/Repository.java | 8 +++++- .../blobstore/BlobStoreRepository.java | 28 ++++++++++++------- 3 files changed, 42 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index 3f6db4ecb4154..070dfa1b0ae17 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -18,10 +18,19 @@ import java.util.function.BooleanSupplier; import java.util.function.Consumer; +/** + * Describes the context of fetching {@link SnapshotInfo} via {@link Repository#getSnapshotInfo(GetSnapshotInfoContext)}. + */ public final class GetSnapshotInfoContext implements ActionListener { + /** + * Snapshot ids to fetch info for + */ private final List snapshotIds; + /** + * Stop fetching additional {@link SnapshotInfo} if an exception is encountered. + */ private final boolean failFast; private volatile boolean failed = false; @@ -32,6 +41,8 @@ public final class GetSnapshotInfoContext implements ActionListener onSnapshotInfo; + // TODO: enhance org.elasticsearch.common.util.concurrent.CountDown to allow for an atomic check and try-countdown and use it to + // simplify the logic here private final AtomicInteger counter; private final AtomicReference exception = new AtomicReference<>(); @@ -67,8 +78,13 @@ public boolean isDone() { @Override public void onResponse(SnapshotInfo snapshotInfo) { + final int updatedCount = counter.decrementAndGet(); + if (updatedCount < 0) { + assert failFast && failed : "must only get here for fail-fast execution that failed"; + return; + } onSnapshotInfo.accept(snapshotInfo); - if (counter.decrementAndGet() == 0) { + if (updatedCount == 0) { try { doneListener.onResponse(null); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 0cd741cd893e0..3ad684ea02cf4 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -71,12 +71,18 @@ default Repository create(RepositoryMetadata metadata, Function listener) { getSnapshotInfo( new GetSnapshotInfoContext( diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 9b67caba0e54f..74123c2f5e986 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1239,22 +1239,30 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo return; } threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { + if (context.isDone()) { + return; + } if (context.isCancelled()) { + queue.clear(); context.onFailure(new TaskCancelledException("task cancelled")); return; } + Exception failure = null; try { - try { - context.onResponse(SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry)); - } catch (NoSuchFileException ex) { - context.onFailure(new SnapshotMissingException(metadata.name(), snapshotId, ex)); - } catch (IOException | NotXContentException ex) { - context.onFailure(new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex)); - } + context.onResponse(SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry)); + } catch (NoSuchFileException ex) { + failure = new SnapshotMissingException(metadata.name(), snapshotId, ex); + } catch (IOException | NotXContentException ex) { + failure = new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex); } catch (Exception e) { - context.onFailure(e instanceof SnapshotException - ? e - : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e)); + failure = e instanceof SnapshotException + ? e : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e); + } + if (failure != null) { + if (context.failFast()) { + queue.clear(); + } + context.onFailure(failure); } getOneSnapshotInfo(queue, context); }); From 4d15949085545b2688455bfb06ed29664ce34160 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 31 May 2021 22:13:35 +0200 Subject: [PATCH 03/14] fix concurrency --- .../status/TransportSnapshotsStatusAction.java | 10 +++++----- .../repositories/GetSnapshotInfoContext.java | 7 +------ 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 81af993bc30be..89b843e9ade63 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -13,6 +13,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.StepListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.node.NodeClient; @@ -26,7 +27,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.util.CollectionUtils; -import org.elasticsearch.common.util.concurrent.ListenableFuture; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -226,10 +226,10 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho List builder, Set currentSnapshotNames, String repositoryName, ActionListener listener) { final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); - final ListenableFuture repositoryDataListener = new ListenableFuture<>(); + final StepListener repositoryDataListener = new StepListener<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); final Collection snapshotIdsToLoad = new ArrayList<>(); - repositoryDataListener.addListener(ActionListener.wrap(repositoryData -> { + repositoryDataListener.whenComplete(repositoryData -> { final Map matchedSnapshotIds = repositoryData.getSnapshotIds().stream() .filter(s -> requestedSnapshotNames.contains(s.getName())) .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); @@ -302,7 +302,7 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho new ActionListener<>() { @Override public void onResponse(Void unused) { - listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); + listener.onResponse(new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))); } @Override @@ -312,7 +312,7 @@ public void onFailure(Exception e) { } )); } - }, listener::onFailure), threadPool.generic(), null); + }, listener::onFailure); } /** diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index 070dfa1b0ae17..105862c74a21a 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -78,13 +78,8 @@ public boolean isDone() { @Override public void onResponse(SnapshotInfo snapshotInfo) { - final int updatedCount = counter.decrementAndGet(); - if (updatedCount < 0) { - assert failFast && failed : "must only get here for fail-fast execution that failed"; - return; - } onSnapshotInfo.accept(snapshotInfo); - if (updatedCount == 0) { + if (counter.decrementAndGet() == 0) { try { doneListener.onResponse(null); } catch (Exception e) { From d6f3e3c9d53282e2574d7b37b9cdf2374fe99395 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 1 Jun 2021 06:45:35 +0200 Subject: [PATCH 04/14] adjust --- .../snapshots/status/TransportSnapshotsStatusAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 89b843e9ade63..f6fe5b6996c4c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -267,7 +267,7 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho try { shardStatuses = snapshotShards(repositoryName, repositoryData, snapshotInfo); } catch (Exception e) { - // TODO: refactor, this is not ok if we run into an exception and resolve the listener twice + // TODO: enhance logic to abort other requests as soon as we fail here listener.onFailure(e); return; } From 0ef6f1b3d4f9fbb67c9f2fbdeceddee0d4f55069 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 1 Jun 2021 12:11:34 +0200 Subject: [PATCH 05/14] snapshot status API fetch logic --- .../get/TransportGetSnapshotsAction.java | 2 +- .../TransportSnapshotsStatusAction.java | 18 ++----- .../repositories/GetSnapshotInfoContext.java | 48 ++++++++++++------- .../repositories/Repository.java | 2 +- .../blobstore/BlobStoreRepository.java | 10 ++-- 5 files changed, 44 insertions(+), 36 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index d86a5f4dc26a8..4424002dcfc9d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -283,7 +283,7 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, snapshotIdsToIterate, ignoreUnavailable == false, task::isCancelled, - snapshotInfos::add, + (context, snapshotInfo) -> snapshotInfos.add(snapshotInfo), ignoreUnavailable ? new ActionListener<>() { @Override public void onResponse(Void unused) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index f6fe5b6996c4c..da64537df34e5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -261,14 +261,14 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho final List threadSafeBuilder = Collections.synchronizedList(builder); repositoriesService.repository(repositoryName) .getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, () -> false, - snapshotInfo -> { + (context, snapshotInfo) -> { List shardStatusBuilder = new ArrayList<>(); final Map shardStatuses; try { shardStatuses = snapshotShards(repositoryName, repositoryData, snapshotInfo); } catch (Exception e) { - // TODO: enhance logic to abort other requests as soon as we fail here - listener.onFailure(e); + // stops all further fetches of snapshotInfo since context is fail-fast + context.onFailure(e); return; } for (Map.Entry shardStatus : shardStatuses.entrySet()) { @@ -299,17 +299,7 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, Snapsho // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime)); }, - new ActionListener<>() { - @Override - public void onResponse(Void unused) { - listener.onResponse(new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - } + listener.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))) )); } }, listener::onFailure); diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index 105862c74a21a..e5e3583a0298e 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -15,8 +15,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; -import java.util.function.Consumer; /** * Describes the context of fetching {@link SnapshotInfo} via {@link Repository#getSnapshotInfo(GetSnapshotInfoContext)}. @@ -39,7 +39,7 @@ public final class GetSnapshotInfoContext implements ActionListener doneListener; - private final Consumer onSnapshotInfo; + private final BiConsumer onSnapshotInfo; // TODO: enhance org.elasticsearch.common.util.concurrent.CountDown to allow for an atomic check and try-countdown and use it to // simplify the logic here @@ -50,7 +50,7 @@ public final class GetSnapshotInfoContext implements ActionListener snapshotIds, boolean failFast, BooleanSupplier isCancelled, - Consumer onSnapshotInfo, + BiConsumer onSnapshotInfo, ActionListener doneListener) { this.snapshotIds = List.copyOf(snapshotIds); this.counter = new AtomicInteger(snapshotIds.size()); @@ -64,21 +64,36 @@ public List snapshotIds() { return snapshotIds; } + /** + * @return true if fetching {@link SnapshotInfo} should be stopped after encountering any exception + */ public boolean failFast() { return failFast; } + /** + * @return true if fetching {@link SnapshotInfo} has been cancelled + */ public boolean isCancelled() { return isCancelled.getAsBoolean(); } - public boolean isDone() { + /** + * @return true if fetching {@link SnapshotInfo} has been stopped + */ + public boolean stopped() { return failed; } @Override public void onResponse(SnapshotInfo snapshotInfo) { - onSnapshotInfo.accept(snapshotInfo); + try { + onSnapshotInfo.accept(this, snapshotInfo); + } catch (Exception e) { + assert false : e; + onFailure(e); + return; + } if (counter.decrementAndGet() == 0) { try { doneListener.onResponse(null); @@ -94,12 +109,7 @@ public void onFailure(Exception e) { if (failFast) { failed = true; if (counter.getAndSet(0) > 0) { - try { - doneListener.onFailure(e); - } catch (Exception ex) { - assert false : ex; - throw ex; - } + failDoneListener(e); } } else { final Exception failure = exception.updateAndGet(ex -> { @@ -110,13 +120,17 @@ public void onFailure(Exception e) { return ex; }); if (counter.decrementAndGet() == 0) { - try { - doneListener.onFailure(failure); - } catch (Exception ex) { - assert false : ex; - throw ex; - } + failDoneListener(failure); } } } + + private void failDoneListener(Exception failure) { + try { + doneListener.onFailure(failure); + } catch (Exception ex) { + assert false : ex; + throw ex; + } + } } diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 3ad684ea02cf4..20eb22231f33d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -89,7 +89,7 @@ default void getSnapshotInfo(SnapshotId snapshotId, ActionListener List.of(snapshotId), true, () -> false, - listener::onResponse, + (context, snapshotInfo) -> listener.onResponse(snapshotInfo), new ActionListener<>() { @Override public void onResponse(Void o) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 74123c2f5e986..fa40a58f7254b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1239,7 +1239,7 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo return; } threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { - if (context.isDone()) { + if (context.stopped()) { return; } if (context.isCancelled()) { @@ -1248,8 +1248,9 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo return; } Exception failure = null; + SnapshotInfo snapshotInfo = null; try { - context.onResponse(SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry)); + snapshotInfo = SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry); } catch (NoSuchFileException ex) { failure = new SnapshotMissingException(metadata.name(), snapshotId, ex); } catch (IOException | NotXContentException ex) { @@ -1263,6 +1264,9 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo queue.clear(); } context.onFailure(failure); + } else { + assert snapshotInfo != null; + context.onResponse(snapshotInfo); } getOneSnapshotInfo(queue, context); }); @@ -1902,7 +1906,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS snapshotIdsWithMissingDetails, false, () -> false, - snapshotInfo -> extraDetailsMap.put(snapshotInfo.snapshotId(), + (context, snapshotInfo) -> extraDetailsMap.put(snapshotInfo.snapshotId(), new SnapshotDetails( snapshotInfo.state(), snapshotInfo.version(), From c02244a8082c96c7e9f37c505c66cd8fab37abf0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 1 Jun 2021 12:40:36 +0200 Subject: [PATCH 06/14] some additonal docs --- .../get/TransportGetSnapshotsAction.java | 2 +- .../repositories/GetSnapshotInfoContext.java | 23 +++++++++++++------ .../blobstore/BlobStoreRepository.java | 2 +- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index 4424002dcfc9d..2da86a779681c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -293,7 +293,7 @@ public void onResponse(Void unused) { @Override public void onFailure(Exception e) { - logger.debug("failed to fetch some snapshot infos", e); + logger.debug("failed to fetch snapshot info for some snapshots", e); allDoneListener.onResponse(null); } } : allDoneListener diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index e5e3583a0298e..f0aed6cb247db 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -33,12 +33,22 @@ public final class GetSnapshotInfoContext implements ActionListener doneListener; + /** + * {@link BiConsumer} invoked for each {@link SnapshotInfo} that is fetched with this instance and the {@code SnapshotInfo} as + * arguments. + */ private final BiConsumer onSnapshotInfo; // TODO: enhance org.elasticsearch.common.util.concurrent.CountDown to allow for an atomic check and try-countdown and use it to @@ -79,10 +89,10 @@ public boolean isCancelled() { } /** - * @return true if fetching {@link SnapshotInfo} has been stopped + * @return true if fetching {@link SnapshotInfo} is either complete or should be stopped because of an error */ - public boolean stopped() { - return failed; + public boolean done() { + return counter.get() <= 0; } @Override @@ -99,7 +109,7 @@ public void onResponse(SnapshotInfo snapshotInfo) { doneListener.onResponse(null); } catch (Exception e) { assert false : e; - doneListener.onFailure(e); + failDoneListener(e); } } } @@ -107,7 +117,6 @@ public void onResponse(SnapshotInfo snapshotInfo) { @Override public void onFailure(Exception e) { if (failFast) { - failed = true; if (counter.getAndSet(0) > 0) { failDoneListener(e); } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index fa40a58f7254b..a8bc856257572 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1239,7 +1239,7 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo return; } threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { - if (context.stopped()) { + if (context.done()) { return; } if (context.isCancelled()) { From 24de437791e1466ae84569a0e7e3720208d5578b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 2 Jun 2021 12:37:19 +0200 Subject: [PATCH 07/14] fix todo --- .../repositories/GetSnapshotInfoContext.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index f0aed6cb247db..8fea6da7cd873 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -8,12 +8,12 @@ package org.elasticsearch.repositories; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import java.util.Collection; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; @@ -51,9 +51,7 @@ public final class GetSnapshotInfoContext implements ActionListener onSnapshotInfo; - // TODO: enhance org.elasticsearch.common.util.concurrent.CountDown to allow for an atomic check and try-countdown and use it to - // simplify the logic here - private final AtomicInteger counter; + private final CountDown counter; private final AtomicReference exception = new AtomicReference<>(); @@ -63,7 +61,7 @@ public GetSnapshotInfoContext(Collection snapshotIds, BiConsumer onSnapshotInfo, ActionListener doneListener) { this.snapshotIds = List.copyOf(snapshotIds); - this.counter = new AtomicInteger(snapshotIds.size()); + this.counter = new CountDown(snapshotIds.size()); this.failFast = failFast; this.isCancelled = isCancelled; this.onSnapshotInfo = onSnapshotInfo; @@ -92,7 +90,7 @@ public boolean isCancelled() { * @return true if fetching {@link SnapshotInfo} is either complete or should be stopped because of an error */ public boolean done() { - return counter.get() <= 0; + return counter.isCountedDown(); } @Override @@ -104,7 +102,7 @@ public void onResponse(SnapshotInfo snapshotInfo) { onFailure(e); return; } - if (counter.decrementAndGet() == 0) { + if (counter.countDown()) { try { doneListener.onResponse(null); } catch (Exception e) { @@ -117,7 +115,7 @@ public void onResponse(SnapshotInfo snapshotInfo) { @Override public void onFailure(Exception e) { if (failFast) { - if (counter.getAndSet(0) > 0) { + if (counter.fastForward()) { failDoneListener(e); } } else { @@ -128,7 +126,7 @@ public void onFailure(Exception e) { ex.addSuppressed(e); return ex; }); - if (counter.decrementAndGet() == 0) { + if (counter.countDown()) { failDoneListener(failure); } } From bc4515ee323307be4592f44b5a1995407c1bcb40 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 2 Jun 2021 14:29:43 +0200 Subject: [PATCH 08/14] cleanup --- .../get/TransportGetSnapshotsAction.java | 9 +++-- .../repositories/GetSnapshotInfoContext.java | 33 +++++++++---------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index 2da86a779681c..ea632f4813315 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -284,19 +284,18 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, ignoreUnavailable == false, task::isCancelled, (context, snapshotInfo) -> snapshotInfos.add(snapshotInfo), - ignoreUnavailable ? new ActionListener<>() { + ignoreUnavailable ? ActionListener.runAfter(new ActionListener<>() { @Override public void onResponse(Void unused) { logger.trace("done fetching snapshot infos [{}]", snapshotIdsToIterate); - allDoneListener.onResponse(null); } @Override public void onFailure(Exception e) { - logger.debug("failed to fetch snapshot info for some snapshots", e); - allDoneListener.onResponse(null); + assert false : new AssertionError("listener should always complete successfully for ignoreUnavailable=true", e); + logger.warn("failed to fetch snapshot info for some snapshots", e); } - } : allDoneListener + }, () -> allDoneListener.onResponse(null)) : allDoneListener ) ); } diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index 8fea6da7cd873..750f32851fa34 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -7,6 +7,8 @@ */ package org.elasticsearch.repositories; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.snapshots.SnapshotId; @@ -14,7 +16,6 @@ import java.util.Collection; import java.util.List; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; @@ -23,15 +24,17 @@ */ public final class GetSnapshotInfoContext implements ActionListener { + private static final Logger logger = LogManager.getLogger(GetSnapshotInfoContext.class); + /** - * Snapshot ids to fetch info for + * Snapshot ids to fetch info for. */ private final List snapshotIds; /** * Stop fetching additional {@link SnapshotInfo} if an exception is encountered. */ - private final boolean failFast; + private final boolean abortOnFailure; /** * If this supplier returns true, indicates that the task that initiated this context has been cancelled and that not further fetching @@ -41,7 +44,8 @@ public final class GetSnapshotInfoContext implements ActionListener doneListener; @@ -53,16 +57,15 @@ public final class GetSnapshotInfoContext implements ActionListener exception = new AtomicReference<>(); - public GetSnapshotInfoContext(Collection snapshotIds, - boolean failFast, + boolean abortOnFailure, BooleanSupplier isCancelled, BiConsumer onSnapshotInfo, ActionListener doneListener) { + assert snapshotIds.isEmpty() == false : "no snapshot ids to fetch given"; this.snapshotIds = List.copyOf(snapshotIds); this.counter = new CountDown(snapshotIds.size()); - this.failFast = failFast; + this.abortOnFailure = abortOnFailure; this.isCancelled = isCancelled; this.onSnapshotInfo = onSnapshotInfo; this.doneListener = doneListener; @@ -76,7 +79,7 @@ public List snapshotIds() { * @return true if fetching {@link SnapshotInfo} should be stopped after encountering any exception */ public boolean failFast() { - return failFast; + return abortOnFailure; } /** @@ -114,20 +117,14 @@ public void onResponse(SnapshotInfo snapshotInfo) { @Override public void onFailure(Exception e) { - if (failFast) { + if (abortOnFailure) { if (counter.fastForward()) { failDoneListener(e); } } else { - final Exception failure = exception.updateAndGet(ex -> { - if (ex == null) { - return e; - } - ex.addSuppressed(e); - return ex; - }); + logger.warn("failed to fetch snapshot info", e); if (counter.countDown()) { - failDoneListener(failure); + doneListener.onResponse(null); } } } From 854f4430f2b71e4a1757ae94ab0e161367ec45d5 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 2 Jun 2021 17:44:41 +0200 Subject: [PATCH 09/14] tests fixed --- .../repositories/hdfs/HdfsTests.java | 3 +- .../BlobStoreRepositoryCleanupIT.java | 2 +- .../snapshots/SnapshotResiliencyTests.java | 13 +- .../AbstractThirdPartyRepositoryTestCase.java | 2 +- .../blobstore/BlobStoreTestUtil.java | 116 ++++++++++++++---- .../AbstractSnapshotIntegTestCase.java | 2 +- 6 files changed, 101 insertions(+), 37 deletions(-) diff --git a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java index 946cfdc7be897..53c9f764879e0 100644 --- a/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java +++ b/plugins/repository-hdfs/src/test/java/org/elasticsearch/repositories/hdfs/HdfsTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.ESSingleNodeTestCase; -import org.elasticsearch.threadpool.ThreadPool; import java.util.Collection; @@ -140,7 +139,7 @@ public void testSimpleWorkflow() { assertThat(clusterState.getMetadata().hasIndex("test-idx-2"), equalTo(false)); final BlobStoreRepository repo = (BlobStoreRepository) getInstanceFromNode(RepositoriesService.class).repository("test-repo"); - BlobStoreTestUtil.assertConsistency(repo, repo.threadPool().executor(ThreadPool.Names.GENERIC)); + BlobStoreTestUtil.assertConsistency(repo); } public void testMissingUri() { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java index 3a22d933c057a..7609cd43ae514 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java @@ -131,6 +131,6 @@ public void testCleanupOldIndexN() throws ExecutionException, InterruptedExcepti logger.info("--> cleanup repository"); client().admin().cluster().prepareCleanupRepository(repoName).get(); - BlobStoreTestUtil.assertConsistency(repository, repository.threadPool().generic()); + BlobStoreTestUtil.assertConsistency(repository); } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 308d0f643df3c..1224af58c46bb 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -162,6 +162,8 @@ import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.script.ScriptCompiler; import org.elasticsearch.script.ScriptService; @@ -266,11 +268,12 @@ public void verifyReposThenStopServices() { if (blobStoreContext != null) { blobStoreContext.forceConsistent(); } - - // TODO: bring this back to life - // BlobStoreTestUtil.assertConsistency( - // (BlobStoreRepository) testClusterNodes.randomMasterNodeSafe().repositoriesService.repository("repo"), - // Runnable::run); + final PlainActionFuture future = BlobStoreTestUtil.assertConsistencyAsync( + (BlobStoreRepository) testClusterNodes.randomMasterNodeSafe().repositoriesService.repository("repo") + ); + deterministicTaskQueue.runAllRunnableTasks(); + assertTrue(future.isDone()); + assertNull(future.actionGet(0)); } finally { testClusterNodes.nodes.values().forEach(TestClusterNodes.TestClusterNode::stop); } diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/AbstractThirdPartyRepositoryTestCase.java b/test/framework/src/main/java/org/elasticsearch/repositories/AbstractThirdPartyRepositoryTestCase.java index f8a26696a24c1..71f02b9129021 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/AbstractThirdPartyRepositoryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/AbstractThirdPartyRepositoryTestCase.java @@ -248,7 +248,7 @@ protected boolean assertCorruptionVisible(BlobStoreRepository repo, Executor exe } protected void assertConsistentRepository(BlobStoreRepository repo, Executor executor) throws Exception { - BlobStoreTestUtil.assertConsistency(repo, executor); + BlobStoreTestUtil.assertConsistency(repo); } protected void assertDeleted(BlobPath path, String name) throws Exception { diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java index 1421f0b2c9150..88ad00c3d3ff9 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java @@ -9,6 +9,7 @@ import org.apache.lucene.util.SameThreadExecutorService; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -33,19 +34,19 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; -import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.ShardGenerations; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; -import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.threadpool.ThreadPool; import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.file.NoSuchFileException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -55,7 +56,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -78,23 +78,25 @@ public final class BlobStoreTestUtil { - public static void assertRepoConsistency(InternalTestCluster testCluster, String repoName) { - final BlobStoreRepository repo = - (BlobStoreRepository) testCluster.getCurrentMasterNodeInstance(RepositoriesService.class).repository(repoName); - BlobStoreTestUtil.assertConsistency(repo, repo.threadPool().executor(ThreadPool.Names.GENERIC)); - } - /** * Assert that there are no unreferenced indices or unreferenced root-level metadata blobs in any repository. * TODO: Expand the logic here to also check for unreferenced segment blobs and shard level metadata * @param repository BlobStoreRepository to check - * @param executor Executor to run all repository calls on. This is needed since the production {@link BlobStoreRepository} - * implementations assert that all IO inducing calls happen on the generic or snapshot thread-pools and hence callers - * of this assertion must pass an executor on those when using such an implementation. */ - public static void assertConsistency(BlobStoreRepository repository, Executor executor) { - final PlainActionFuture listener = PlainActionFuture.newFuture(); - executor.execute(ActionRunnable.supply(listener, () -> { + public static void assertConsistency(BlobStoreRepository repository) { + final PlainActionFuture listener = assertConsistencyAsync(repository); + final AssertionError err = listener.actionGet(TimeValue.timeValueMinutes(1L)); + if (err != null) { + throw new AssertionError(err); + } + } + + /** + * Same as {@link #assertConsistency(BlobStoreRepository)} but async so it can be used in tests that don't allow blocking. + */ + public static PlainActionFuture assertConsistencyAsync(BlobStoreRepository repository) { + final PlainActionFuture future = PlainActionFuture.newFuture(); + repository.threadPool().generic().execute(ActionRunnable.wrap(future, listener -> { try { final BlobContainer blobContainer = repository.blobContainer(); final long latestGen; @@ -111,17 +113,37 @@ public static void assertConsistency(BlobStoreRepository repository, Executor ex repositoryData = RepositoryData.snapshotsFromXContent(parser, latestGen, false); } assertIndexUUIDs(repository, repositoryData); - assertSnapshotUUIDs(repository, repositoryData); - assertShardIndexGenerations(blobContainer, repositoryData.shardGenerations()); - return null; + assertSnapshotUUIDs(repository, repositoryData, new ActionListener<>() { + @Override + public void onResponse(AssertionError assertionError) { + if (assertionError == null) { + try { + try { + assertShardIndexGenerations(blobContainer, repositoryData.shardGenerations()); + } catch (AssertionError e) { + listener.onResponse(e); + return; + } + } catch (Exception e) { + onFailure(e); + return; + } + listener.onResponse(null); + } else { + listener.onResponse(assertionError); + } + } + + @Override + public void onFailure(Exception e) { + listener.onResponse(new AssertionError(e)); + } + }); } catch (AssertionError e) { - return e; + listener.onResponse(e); } })); - final AssertionError err = listener.actionGet(TimeValue.timeValueMinutes(1L)); - if (err != null) { - throw new AssertionError(err); - } + return future; } private static void assertIndexGenerations(BlobContainer repoRoot, long latestGen) throws IOException { @@ -183,7 +205,8 @@ private static void assertIndexUUIDs(BlobStoreRepository repository, RepositoryD } } - private static void assertSnapshotUUIDs(BlobStoreRepository repository, RepositoryData repositoryData) throws IOException { + private static void assertSnapshotUUIDs(BlobStoreRepository repository, RepositoryData repositoryData, + ActionListener listener) throws IOException { final BlobContainer repoRoot = repository.blobContainer(); final Collection snapshotIds = repositoryData.getSnapshotIds(); final List expectedSnapshotUUIDs = snapshotIds.stream().map(SnapshotId::getUUID).collect(Collectors.toList()); @@ -201,11 +224,50 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito } else { indices = indicesContainer.children(); } + if (snapshotIds.isEmpty()) { + listener.onResponse(null); + return; + } + // Assert that for each snapshot, the relevant metadata was written to index and shard folders + final List snapshotInfos = Collections.synchronizedList(new ArrayList<>()); + repository.getSnapshotInfo( + new GetSnapshotInfoContext( + List.copyOf(snapshotIds), + true, + () -> false, + (ctx, sni) -> snapshotInfos.add(sni), + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + try { + assertSnapshotInfosConsistency(repository, repositoryData, indices, snapshotInfos); + } catch (Exception e) { + listener.onResponse(new AssertionError(e)); + return; + } catch (AssertionError e) { + listener.onResponse(e); + return; + } + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + listener.onResponse(new AssertionError(e)); + } + } + ) + ); + } + + private static void assertSnapshotInfosConsistency(BlobStoreRepository repository, + RepositoryData repositoryData, + Map indices, + List snapshotInfos) throws IOException { final Map maxShardCountsExpected = new HashMap<>(); final Map maxShardCountsSeen = new HashMap<>(); - // Assert that for each snapshot, the relevant metadata was written to index and shard folders - for (SnapshotId snapshotId: snapshotIds) { - final SnapshotInfo snapshotInfo = PlainActionFuture.get(f -> repository.getSnapshotInfo(snapshotId, f)); + for (SnapshotInfo snapshotInfo: snapshotInfos) { + final SnapshotId snapshotId = snapshotInfo.snapshotId(); for (String index : snapshotInfo.indices()) { final IndexId indexId = repositoryData.resolveIndexId(index); assertThat(indices, hasKey(indexId.getId())); diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index e173025d52ded..a75a87c56a59a 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -134,7 +134,7 @@ public void assertRepoConsistency() { clusterAdmin().prepareDeleteSnapshot(name, OLD_VERSION_SNAPSHOT_PREFIX + "*").get(); clusterAdmin().prepareCleanupRepository(name).get(); } - BlobStoreTestUtil.assertRepoConsistency(internalCluster(), name); + BlobStoreTestUtil.assertConsistency(getRepositoryOnMaster(name)); }); } else { logger.info("--> skipped repo consistency checks because [{}]", skipRepoConsistencyCheckReason); From df02ed90a64f466f508bb7ec99123507b5380362 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 10 Jun 2021 10:29:48 +0200 Subject: [PATCH 10/14] spotless --- .../groovy/elasticsearch.formatting.gradle | 76 +- .../repositories/RepositoryBlocksIT.java | 48 +- .../cluster/snapshots/SnapshotBlocksIT.java | 82 +- .../repositories/RepositoriesServiceIT.java | 29 +- .../blobstore/BlobStoreDynamicSettingsIT.java | 47 +- .../BlobStoreRepositoryCleanupIT.java | 65 +- .../blobstore/BlobStoreSizeLimitIT.java | 20 +- .../fs/FsBlobStoreRepositoryIntegTests.java | 4 +- .../snapshots/AbortedRestoreIT.java | 16 +- .../snapshots/BlobStoreIncrementalityIT.java | 39 +- .../snapshots/CloneSnapshotIT.java | 173 +- .../snapshots/ConcurrentSnapshotsIT.java | 296 ++- .../CorruptedBlobStoreRepositoryIT.java | 514 ++++-- .../snapshots/CustomMetadataSnapshotIT.java | 93 +- .../DedicatedClusterSnapshotRestoreIT.java | 419 +++-- .../snapshots/FeatureStateResetApiIT.java | 57 +- ...etadataLoadingDuringSnapshotRestoreIT.java | 77 +- .../snapshots/MultiClusterRepoAccessIT.java | 151 +- .../snapshots/RepositoriesIT.java | 116 +- .../RepositoryFilterUserMetadataIT.java | 62 +- .../snapshots/RestoreSnapshotIT.java | 476 +++-- .../SharedClusterSnapshotRestoreIT.java | 962 ++++++---- .../snapshots/SnapshotBrokenSettingsIT.java | 53 +- .../SnapshotCustomPluginStateIT.java | 85 +- .../snapshots/SnapshotShardsServiceIT.java | 9 +- .../snapshots/SnapshotStatusApisIT.java | 274 +-- .../snapshots/SystemIndicesSnapshotIT.java | 124 +- .../CleanupRepositoryRequestBuilder.java | 10 +- .../cleanup/CleanupRepositoryResponse.java | 17 +- .../TransportCleanupRepositoryAction.java | 155 +- .../delete/DeleteRepositoryAction.java | 1 - .../delete/DeleteRepositoryRequest.java | 3 +- .../DeleteRepositoryRequestBuilder.java | 6 +- .../TransportDeleteRepositoryAction.java | 37 +- .../get/GetRepositoriesAction.java | 1 - .../get/GetRepositoriesRequest.java | 3 +- .../get/GetRepositoriesRequestBuilder.java | 6 +- .../get/GetRepositoriesResponse.java | 1 - .../get/TransportGetRepositoriesAction.java | 31 +- .../repositories/put/PutRepositoryAction.java | 1 - .../put/PutRepositoryRequest.java | 3 +- .../put/PutRepositoryRequestBuilder.java | 6 +- .../put/TransportPutRepositoryAction.java | 31 +- .../TransportVerifyRepositoryAction.java | 42 +- .../verify/VerifyRepositoryAction.java | 1 - .../verify/VerifyRepositoryRequest.java | 3 +- .../VerifyRepositoryRequestBuilder.java | 6 +- .../verify/VerifyRepositoryResponse.java | 28 +- .../clone/CloneSnapshotRequestBuilder.java | 22 +- .../clone/TransportCloneSnapshotAction.java | 31 +- .../create/CreateSnapshotAction.java | 1 - .../create/CreateSnapshotRequest.java | 72 +- .../create/CreateSnapshotRequestBuilder.java | 6 +- .../create/CreateSnapshotResponse.java | 18 +- .../create/TransportCreateSnapshotAction.java | 32 +- .../delete/DeleteSnapshotAction.java | 1 - .../delete/DeleteSnapshotRequest.java | 4 +- .../delete/DeleteSnapshotRequestBuilder.java | 6 +- .../delete/TransportDeleteSnapshotAction.java | 31 +- .../features/ResetFeatureStateRequest.java | 3 +- .../features/ResetFeatureStateResponse.java | 24 +- .../TransportResetFeatureStateAction.java | 8 +- .../TransportSnapshottableFeaturesAction.java | 55 +- .../snapshots/get/GetSnapshotsAction.java | 1 - .../snapshots/get/GetSnapshotsRequest.java | 12 +- .../get/GetSnapshotsRequestBuilder.java | 8 +- .../snapshots/get/GetSnapshotsResponse.java | 46 +- .../get/TransportGetSnapshotsAction.java | 195 +- .../restore/RestoreClusterStateListener.java | 21 +- .../restore/RestoreSnapshotAction.java | 1 - .../restore/RestoreSnapshotRequest.java | 63 +- .../RestoreSnapshotRequestBuilder.java | 9 +- .../restore/RestoreSnapshotResponse.java | 13 +- .../TransportRestoreSnapshotAction.java | 32 +- .../status/SnapshotIndexShardStage.java | 11 +- .../status/SnapshotIndexShardStatus.java | 26 +- .../snapshots/status/SnapshotIndexStatus.java | 28 +- .../snapshots/status/SnapshotShardsStats.java | 13 +- .../snapshots/status/SnapshotStats.java | 25 +- .../snapshots/status/SnapshotStatus.java | 48 +- .../status/SnapshotsStatusAction.java | 1 - .../status/SnapshotsStatusRequest.java | 3 +- .../status/SnapshotsStatusRequestBuilder.java | 6 +- .../status/SnapshotsStatusResponse.java | 6 +- .../status/TransportNodesSnapshotsStatus.java | 43 +- .../TransportSnapshotsStatusAction.java | 238 ++- .../snapshots/IndexShardRestoreException.java | 2 +- .../IndexShardRestoreFailedException.java | 2 +- .../IndexShardSnapshotException.java | 2 +- .../IndexShardSnapshotFailedException.java | 2 +- .../snapshots/IndexShardSnapshotStatus.java | 140 +- .../BlobStoreIndexShardSnapshot.java | 42 +- .../BlobStoreIndexShardSnapshots.java | 9 +- .../snapshots/blobstore/SnapshotFiles.java | 11 +- .../repositories/FilterRepository.java | 64 +- .../repositories/GetSnapshotInfoContext.java | 12 +- .../IndexMetaDataGenerations.java | 28 +- .../repositories/RepositoriesModule.java | 50 +- .../repositories/RepositoriesService.java | 193 +- .../RepositoriesStatsArchive.java | 11 +- .../repositories/Repository.java | 82 +- .../repositories/RepositoryCleanupResult.java | 7 +- .../repositories/RepositoryData.java | 407 ++-- .../repositories/RepositoryException.java | 2 +- .../repositories/RepositoryInfo.java | 32 +- .../RepositoryMissingException.java | 3 +- .../repositories/RepositoryStats.java | 4 +- .../repositories/RepositoryStatsSnapshot.java | 13 +- .../RepositoryVerificationException.java | 4 +- .../repositories/ShardGenerations.java | 27 +- .../repositories/ShardSnapshotResult.java | 6 +- .../repositories/SnapshotShardContext.java | 21 +- .../VerifyNodeRepositoryAction.java | 31 +- .../blobstore/BlobStoreRepository.java | 1635 +++++++++++------ .../blobstore/ChecksumBlobStoreFormat.java | 43 +- .../blobstore/FileRestoreContext.java | 57 +- .../blobstore/MeteredBlobStoreRepository.java | 22 +- .../repositories/fs/FsRepository.java | 71 +- .../InFlightShardSnapshotStates.java | 22 +- .../InternalSnapshotsInfoService.java | 42 +- .../InvalidSnapshotNameException.java | 1 - .../elasticsearch/snapshots/RestoreInfo.java | 13 +- .../snapshots/RestoreService.java | 653 ++++--- .../snapshots/SnapshotFeatureInfo.java | 12 +- .../SnapshotInProgressException.java | 1 - .../elasticsearch/snapshots/SnapshotInfo.java | 426 +++-- .../snapshots/SnapshotShardFailure.java | 33 +- .../snapshots/SnapshotShardSizeInfo.java | 13 +- .../snapshots/SnapshotShardsService.java | 165 +- .../snapshots/SnapshotState.java | 3 - .../snapshots/SnapshotsService.java | 1563 ++++++++++------ .../verify/VerifyRepositoryResponseTests.java | 2 +- .../create/CreateSnapshotRequestTests.java | 17 +- .../create/CreateSnapshotResponseTests.java | 23 +- ...GetSnapshottableFeaturesResponseTests.java | 30 +- .../ResetFeatureStateResponseTests.java | 33 +- .../get/GetSnapshotsResponseTests.java | 43 +- .../restore/RestoreSnapshotRequestTests.java | 25 +- .../status/SnapshotIndexStatusTests.java | 1 - .../snapshots/status/SnapshotStatsTests.java | 20 +- .../snapshots/status/SnapshotStatusTests.java | 174 +- .../snapshots/blobstore/FileInfoTests.java | 26 +- .../blobstore/SlicedInputStreamTests.java | 24 +- .../repositories/IndexIdTests.java | 2 +- .../repositories/RepositoriesModuleTests.java | 74 +- .../RepositoriesServiceTests.java | 105 +- .../RepositoriesStatsArchiveTests.java | 33 +- .../repositories/RepositoryDataTests.java | 221 ++- .../BlobStoreRepositoryRestoreTests.java | 70 +- .../blobstore/BlobStoreRepositoryTests.java | 170 +- .../repositories/fs/FsRepositoryTests.java | 115 +- .../snapshots/BlobStoreFormatTests.java | 6 +- .../InternalSnapshotsInfoServiceTests.java | 203 +- ...epositoriesMetadataSerializationTests.java | 7 +- .../snapshots/RestoreServiceTests.java | 20 +- ...dSnapshotResultWireSerializationTests.java | 30 +- ...dSnapshotStatusWireSerializationTests.java | 48 +- .../snapshots/SnapshotFeatureInfoTests.java | 15 +- .../SnapshotInfoBlobSerializationTests.java | 13 +- .../snapshots/SnapshotInfoTestUtils.java | 340 ++-- .../snapshots/SnapshotRequestsTests.java | 6 +- .../snapshots/SnapshotResiliencyTests.java | 1122 +++++++---- ...napshotShardFailureSerializationTests.java | 79 +- .../snapshots/SnapshotShardsServiceTests.java | 37 +- .../snapshots/SnapshotUtilsTests.java | 49 +- ...SnapshotsInProgressSerializationTests.java | 254 ++- .../snapshots/SnapshotsServiceTests.java | 215 ++- .../MockEventuallyConsistentRepository.java | 138 +- ...ckEventuallyConsistentRepositoryTests.java | 180 +- 169 files changed, 10290 insertions(+), 5584 deletions(-) diff --git a/build-tools-internal/src/main/groovy/elasticsearch.formatting.gradle b/build-tools-internal/src/main/groovy/elasticsearch.formatting.gradle index 60cebf51d1b40..271a495d2d775 100644 --- a/build-tools-internal/src/main/groovy/elasticsearch.formatting.gradle +++ b/build-tools-internal/src/main/groovy/elasticsearch.formatting.gradle @@ -6,7 +6,7 @@ * Side Public License, v 1. */ -import org.elasticsearch.gradle.internal.BuildPlugin +import org.elasticsearch.gradle.internal.ElasticsearchJavaPlugin /* * This script plugin configures formatting for Java source using Spotless @@ -86,7 +86,9 @@ def projectPathsToExclude = [ ':plugins:analysis-ukrainian', ':plugins:discovery-azure-classic', ':plugins:discovery-ec2', + ':plugins:discovery-ec2:qa:amazon-ec2', ':plugins:discovery-gce', + ':plugins:discovery-gce:qa:gce', ':plugins:ingest-attachment', ':plugins:mapper-annotated-text', ':plugins:mapper-murmur3', @@ -94,75 +96,135 @@ def projectPathsToExclude = [ ':plugins:repository-azure', ':plugins:repository-gcs', ':plugins:repository-hdfs', + ':plugins:repository-hdfs:hadoop-common', ':plugins:repository-s3', ':plugins:store-smb', ':plugins:transport-nio', ':qa:die-with-dignity', ':rest-api-spec', - ':server', ':test:fixtures:azure-fixture', ':test:fixtures:gcs-fixture', + ':test:fixtures:geoip-fixture', ':test:fixtures:hdfs-fixture', ':test:fixtures:krb5kdc-fixture', ':test:fixtures:minio-fixture', ':test:fixtures:old-elasticsearch', ':test:fixtures:s3-fixture', + ':test:fixtures:url-fixture', ':test:framework', ':test:logger-usage', ':x-pack:license-tools', + ':x-pack:plugin', ':x-pack:plugin:analytics', ':x-pack:plugin:async-search', ':x-pack:plugin:async-search:qa', + ':x-pack:plugin:async-search:qa:security', + ':x-pack:plugin:autoscaling:qa:rest', ':x-pack:plugin:ccr', ':x-pack:plugin:ccr:qa', + ':x-pack:plugin:ccr:qa:rest', ':x-pack:plugin:core', + ':x-pack:plugin:data-streams:qa:multi-node', + ':x-pack:plugin:data-streams:qa:rest', ':x-pack:plugin:deprecation', ':x-pack:plugin:enrich:qa:common', + ':x-pack:plugin:enrich:qa:rest', + ':x-pack:plugin:enrich:qa:rest-with-advanced-security', + ':x-pack:plugin:enrich:qa:rest-with-security', ':x-pack:plugin:eql', ':x-pack:plugin:eql:qa', ':x-pack:plugin:eql:qa:common', + ':x-pack:plugin:eql:qa:rest', + ':x-pack:plugin:eql:qa:security', + ':x-pack:plugin:fleet:qa:rest', ':x-pack:plugin:frozen-indices', ':x-pack:plugin:graph', + ':x-pack:plugin:graph:qa:with-security', ':x-pack:plugin:identity-provider', + ':x-pack:plugin:identity-provider:qa:idp-rest-tests', ':x-pack:plugin:ilm', + ':x-pack:plugin:ilm:qa:multi-node', + ':x-pack:plugin:ilm:qa:rest', + ':x-pack:plugin:ilm:qa:with-security', ':x-pack:plugin:mapper-constant-keyword', ':x-pack:plugin:mapper-flattened', ':x-pack:plugin:ml', + ':x-pack:plugin:ml:qa:basic-multi-node', + ':x-pack:plugin:ml:qa:disabled', + ':x-pack:plugin:ml:qa:ml-with-security', + ':x-pack:plugin:ml:qa:native-multi-node-tests', + ':x-pack:plugin:ml:qa:single-node-tests', ':x-pack:plugin:monitoring', ':x-pack:plugin:ql', + ':x-pack:plugin:repository-encrypted:qa:azure', + ':x-pack:plugin:repository-encrypted:qa:gcs', + ':x-pack:plugin:repository-encrypted:qa:s3', ':x-pack:plugin:rollup', + ':x-pack:plugin:rollup:qa:rest', ':x-pack:plugin:search-business-rules', + ':x-pack:plugin:searchable-snapshots:qa:rest', ':x-pack:plugin:security', ':x-pack:plugin:security:cli', + ':x-pack:plugin:security:qa:basic-enable-security', + ':x-pack:plugin:security:qa:security-basic', + ':x-pack:plugin:security:qa:security-disabled', + ':x-pack:plugin:security:qa:security-not-enabled', + ':x-pack:plugin:security:qa:security-trial', + ':x-pack:plugin:security:qa:service-account', + ':x-pack:plugin:security:qa:smoke-test-all-realms', + ':x-pack:plugin:security:qa:tls-basic', + ':x-pack:plugin:shutdown:qa:multi-node', ':x-pack:plugin:spatial', ':x-pack:plugin:sql', ':x-pack:plugin:sql:jdbc', ':x-pack:plugin:sql:qa', + ':x-pack:plugin:sql:qa:jdbc', + ':x-pack:plugin:sql:qa:jdbc:security', ':x-pack:plugin:sql:qa:security', ':x-pack:plugin:sql:sql-action', ':x-pack:plugin:sql:sql-cli', ':x-pack:plugin:sql:sql-client', ':x-pack:plugin:sql:sql-proto', + ':x-pack:plugin:stack:qa:rest', + ':x-pack:plugin:text-structure:qa:text-structure-with-security', ':x-pack:plugin:transform', + ':x-pack:plugin:transform:qa:multi-node-tests', + ':x-pack:plugin:transform:qa:single-node-tests', ':x-pack:plugin:vectors', ':x-pack:plugin:watcher', + ':x-pack:plugin:watcher:qa:rest', + ':x-pack:plugin:watcher:qa:with-monitoring', + ':x-pack:plugin:watcher:qa:with-security', ':x-pack:plugin:wildcard', ':x-pack:qa', + ':x-pack:qa:runtime-fields:core-with-mapped', + ':x-pack:qa:runtime-fields:core-with-search', + ':x-pack:qa:runtime-fields:with-security', ':x-pack:qa:security-example-spi-extension', ':x-pack:test:idp-fixture', ':x-pack:test:smb-fixture' ] subprojects { - plugins.withType(BuildPlugin).whenPluginAdded { + plugins.withType(ElasticsearchJavaPlugin).whenPluginAdded { if (projectPathsToExclude.contains(project.path) == false) { project.apply plugin: "com.diffplug.spotless" spotless { java { - // Normally this isn't necessary, but we have Java sources in - // non-standard places - target 'src/**/*.java' + if (project.path == ':server') { + target 'src/*/java/org/elasticsearch/action/admin/cluster/repositories/**/*.java', + 'src/*/java/org/elasticsearch/action/admin/cluster/snapshots/**/*.java', + 'src/*/java/org/elasticsearch/index/snapshots/**/*.java', + 'src/*/java/org/elasticsearch/repositories/**/*.java', + 'src/*/java/org/elasticsearch/snapshots/**/*.java' + + targetExclude 'src/main/java/org/elasticsearch/search/aggregations/metrics/HyperLogLogPlusPlus.java' + } else { + // Normally this isn't necessary, but we have Java sources in + // non-standard places + target 'src/**/*.java' + } removeUnusedImports() eclipse().configFile rootProject.file('build-tools-internal/formatterConfig.xml') @@ -175,7 +237,7 @@ subprojects { } } - tasks.named("precommit").configure {dependsOn 'spotlessJavaCheck' } + tasks.named("precommit").configure { dependsOn 'spotlessJavaCheck' } } } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/repositories/RepositoryBlocksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/repositories/RepositoryBlocksIT.java index 29fc8d10b53d3..aacf50f37910c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/repositories/RepositoryBlocksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/repositories/RepositoryBlocksIT.java @@ -31,32 +31,48 @@ public void testPutRepositoryWithBlocks() { logger.info("--> registering a repository is blocked when the cluster is read only"); try { setClusterReadOnly(true); - assertBlocked(client().admin().cluster().preparePutRepository("test-repo-blocks") + assertBlocked( + client().admin() + .cluster() + .preparePutRepository("test-repo-blocks") .setType("fs") .setVerify(false) - .setSettings(Settings.builder().put("location", randomRepoPath())), Metadata.CLUSTER_READ_ONLY_BLOCK); + .setSettings(Settings.builder().put("location", randomRepoPath())), + Metadata.CLUSTER_READ_ONLY_BLOCK + ); } finally { setClusterReadOnly(false); } logger.info("--> registering a repository is allowed when the cluster is not read only"); - assertAcked(client().admin().cluster().preparePutRepository("test-repo-blocks") + assertAcked( + client().admin() + .cluster() + .preparePutRepository("test-repo-blocks") .setType("fs") .setVerify(false) - .setSettings(Settings.builder().put("location", randomRepoPath()))); + .setSettings(Settings.builder().put("location", randomRepoPath())) + ); } public void testVerifyRepositoryWithBlocks() { - assertAcked(client().admin().cluster().preparePutRepository("test-repo-blocks") + assertAcked( + client().admin() + .cluster() + .preparePutRepository("test-repo-blocks") .setType("fs") .setVerify(false) - .setSettings(Settings.builder().put("location", randomRepoPath()))); + .setSettings(Settings.builder().put("location", randomRepoPath())) + ); // This test checks that the Get Repository operation is never blocked, even if the cluster is read only. try { setClusterReadOnly(true); - VerifyRepositoryResponse response = client().admin().cluster() - .prepareVerifyRepository("test-repo-blocks").execute().actionGet(); + VerifyRepositoryResponse response = client().admin() + .cluster() + .prepareVerifyRepository("test-repo-blocks") + .execute() + .actionGet(); assertThat(response.getNodes().size(), equalTo(cluster().numDataAndMasterNodes())); } finally { setClusterReadOnly(false); @@ -64,10 +80,14 @@ public void testVerifyRepositoryWithBlocks() { } public void testDeleteRepositoryWithBlocks() { - assertAcked(client().admin().cluster().preparePutRepository("test-repo-blocks") + assertAcked( + client().admin() + .cluster() + .preparePutRepository("test-repo-blocks") .setType("fs") .setVerify(false) - .setSettings(Settings.builder().put("location", randomRepoPath()))); + .setSettings(Settings.builder().put("location", randomRepoPath())) + ); logger.info("--> deleting a repository is blocked when the cluster is read only"); try { @@ -82,10 +102,14 @@ public void testDeleteRepositoryWithBlocks() { } public void testGetRepositoryWithBlocks() { - assertAcked(client().admin().cluster().preparePutRepository("test-repo-blocks") + assertAcked( + client().admin() + .cluster() + .preparePutRepository("test-repo-blocks") .setType("fs") .setVerify(false) - .setSettings(Settings.builder().put("location", randomRepoPath()))); + .setSettings(Settings.builder().put("location", randomRepoPath())) + ); // This test checks that the Get Repository operation is never blocked, even if the cluster is read only. try { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java index dfa7c6b35fa65..0fa3cf33b11f0 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java @@ -55,19 +55,26 @@ protected void setUpRepository() throws Exception { logger.info("--> register a repository"); - assertAcked(client().admin().cluster().preparePutRepository(REPOSITORY_NAME) + assertAcked( + client().admin() + .cluster() + .preparePutRepository(REPOSITORY_NAME) .setType("fs") - .setSettings(Settings.builder().put("location", randomRepoPath()))); + .setSettings(Settings.builder().put("location", randomRepoPath())) + ); logger.info("--> verify the repository"); VerifyRepositoryResponse verifyResponse = client().admin().cluster().prepareVerifyRepository(REPOSITORY_NAME).get(); assertThat(verifyResponse.getNodes().size(), equalTo(cluster().numDataAndMasterNodes())); logger.info("--> create a snapshot"); - CreateSnapshotResponse snapshotResponse = client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME) - .setIncludeGlobalState(true) - .setWaitForCompletion(true) - .execute().actionGet(); + CreateSnapshotResponse snapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME) + .setIncludeGlobalState(true) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(snapshotResponse.status(), equalTo(RestStatus.OK)); ensureSearchable(); } @@ -76,16 +83,21 @@ public void testCreateSnapshotWithBlocks() { logger.info("--> creating a snapshot is allowed when the cluster is read only"); try { setClusterReadOnly(true); - assertThat(client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-1") - .setWaitForCompletion(true).get().status(), equalTo(RestStatus.OK)); + assertThat( + client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-1").setWaitForCompletion(true).get().status(), + equalTo(RestStatus.OK) + ); } finally { setClusterReadOnly(false); } logger.info("--> creating a snapshot is allowed when the cluster is not read only"); - CreateSnapshotResponse response = client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-2") - .setWaitForCompletion(true) - .execute().actionGet(); + CreateSnapshotResponse response = client().admin() + .cluster() + .prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-2") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(response.status(), equalTo(RestStatus.OK)); } @@ -93,8 +105,16 @@ public void testCreateSnapshotWithIndexBlocks() { logger.info("--> creating a snapshot is not blocked when an index is read only"); try { enableIndexBlock(INDEX_NAME, SETTING_READ_ONLY); - assertThat(client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-1") - .setIndices(COMMON_INDEX_NAME_MASK).setWaitForCompletion(true).get().status(), equalTo(RestStatus.OK)); + assertThat( + client().admin() + .cluster() + .prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-1") + .setIndices(COMMON_INDEX_NAME_MASK) + .setWaitForCompletion(true) + .get() + .status(), + equalTo(RestStatus.OK) + ); } finally { disableIndexBlock(INDEX_NAME, SETTING_READ_ONLY); } @@ -102,8 +122,16 @@ public void testCreateSnapshotWithIndexBlocks() { logger.info("--> creating a snapshot is blocked when an index is blocked for reads"); try { enableIndexBlock(INDEX_NAME, SETTING_BLOCKS_READ); - assertThat(client().admin().cluster().prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-2") - .setIndices(COMMON_INDEX_NAME_MASK).setWaitForCompletion(true).get().status(), equalTo(RestStatus.OK)); + assertThat( + client().admin() + .cluster() + .prepareCreateSnapshot(REPOSITORY_NAME, "snapshot-2") + .setIndices(COMMON_INDEX_NAME_MASK) + .setWaitForCompletion(true) + .get() + .status(), + equalTo(RestStatus.OK) + ); } finally { disableIndexBlock(INDEX_NAME, SETTING_BLOCKS_READ); } @@ -127,16 +155,21 @@ public void testRestoreSnapshotWithBlocks() { logger.info("--> restoring a snapshot is blocked when the cluster is read only"); try { setClusterReadOnly(true); - assertBlocked(client().admin().cluster().prepareRestoreSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME), - Metadata.CLUSTER_READ_ONLY_BLOCK); + assertBlocked( + client().admin().cluster().prepareRestoreSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME), + Metadata.CLUSTER_READ_ONLY_BLOCK + ); } finally { setClusterReadOnly(false); } logger.info("--> creating a snapshot is allowed when the cluster is not read only"); - RestoreSnapshotResponse response = client().admin().cluster().prepareRestoreSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME) - .setWaitForCompletion(true) - .execute().actionGet(); + RestoreSnapshotResponse response = client().admin() + .cluster() + .prepareRestoreSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(response.status(), equalTo(RestStatus.OK)); assertTrue(indexExists(INDEX_NAME)); assertTrue(indexExists(OTHER_INDEX_NAME)); @@ -158,9 +191,12 @@ public void testSnapshotStatusWithBlocks() { // This test checks that the Snapshot Status operation is never blocked, even if the cluster is read only. try { setClusterReadOnly(true); - SnapshotsStatusResponse response = client().admin().cluster().prepareSnapshotStatus(REPOSITORY_NAME) - .setSnapshots(SNAPSHOT_NAME) - .execute().actionGet(); + SnapshotsStatusResponse response = client().admin() + .cluster() + .prepareSnapshotStatus(REPOSITORY_NAME) + .setSnapshots(SNAPSHOT_NAME) + .execute() + .actionGet(); assertThat(response.getSnapshots(), hasSize(1)); assertThat(response.getSnapshots().get(0).getState().completed(), equalTo(true)); } finally { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/RepositoriesServiceIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/RepositoriesServiceIT.java index c60ae8eb43402..980751bf9c1b9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/RepositoriesServiceIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/RepositoriesServiceIT.java @@ -41,18 +41,18 @@ public void testUpdateRepository() { final String repositoryName = "test-repo"; final Client client = client(); - final RepositoriesService repositoriesService = - cluster.getDataOrMasterNodeInstances(RepositoriesService.class).iterator().next(); + final RepositoriesService repositoriesService = cluster.getDataOrMasterNodeInstances(RepositoriesService.class).iterator().next(); final Settings.Builder repoSettings = Settings.builder().put("location", randomRepoPath()); - assertAcked(client.admin().cluster().preparePutRepository(repositoryName) - .setType(FsRepository.TYPE) - .setSettings(repoSettings) - .get()); + assertAcked( + client.admin().cluster().preparePutRepository(repositoryName).setType(FsRepository.TYPE).setSettings(repoSettings).get() + ); - final GetRepositoriesResponse originalGetRepositoriesResponse = - client.admin().cluster().prepareGetRepositories(repositoryName).get(); + final GetRepositoriesResponse originalGetRepositoriesResponse = client.admin() + .cluster() + .prepareGetRepositories(repositoryName) + .get(); assertThat(originalGetRepositoriesResponse.repositories(), hasSize(1)); RepositoryMetadata originalRepositoryMetadata = originalGetRepositoriesResponse.repositories().get(0); @@ -65,13 +65,14 @@ public void testUpdateRepository() { final boolean updated = randomBoolean(); final String updatedRepositoryType = updated ? "mock" : FsRepository.TYPE; - assertAcked(client.admin().cluster().preparePutRepository(repositoryName) - .setType(updatedRepositoryType) - .setSettings(repoSettings) - .get()); + assertAcked( + client.admin().cluster().preparePutRepository(repositoryName).setType(updatedRepositoryType).setSettings(repoSettings).get() + ); - final GetRepositoriesResponse updatedGetRepositoriesResponse = - client.admin().cluster().prepareGetRepositories(repositoryName).get(); + final GetRepositoriesResponse updatedGetRepositoriesResponse = client.admin() + .cluster() + .prepareGetRepositories(repositoryName) + .get(); assertThat(updatedGetRepositoriesResponse.repositories(), hasSize(1)); final RepositoryMetadata updatedRepositoryMetadata = updatedGetRepositoriesResponse.repositories().get(0); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreDynamicSettingsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreDynamicSettingsIT.java index 50371691007c6..595a59b7c38f6 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreDynamicSettingsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreDynamicSettingsIT.java @@ -52,18 +52,20 @@ public void testUpdateRateLimitsDynamically() throws Exception { assertNull(currentSettings.get(BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC.getKey())); createRepository( - repoName, - "mock", - Settings.builder().put(currentSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1b"), - randomBoolean() + repoName, + "mock", + Settings.builder().put(currentSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1b"), + randomBoolean() ); assertSame(repoOnMaster, getRepositoryOnNode(repoName, masterNode)); assertSame(repoOnDataNode, getRepositoryOnNode(repoName, dataNode)); final Settings updatedSettings = repoOnMaster.getMetadata().settings(); - assertEquals(ByteSizeValue.ofBytes(1L), - updatedSettings.getAsBytesSize(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), ByteSizeValue.ZERO)); + assertEquals( + ByteSizeValue.ofBytes(1L), + updatedSettings.getAsBytesSize(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), ByteSizeValue.ZERO) + ); assertNull(currentSettings.get(BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC.getKey())); final ActionFuture snapshot1 = startFullSnapshotBlockedOnDataNode("snapshot-2", repoName, dataNode); @@ -71,10 +73,10 @@ public void testUpdateRateLimitsDynamically() throws Exception { // we only run concurrent verification when we have a large SNAPSHOT pool on the data node because otherwise the verification would // deadlock since the small pool is already blocked by the snapshot on the data node createRepository( - repoName, - "mock", - Settings.builder().put(updatedSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1024b"), - largeSnapshotPool && randomBoolean() + repoName, + "mock", + Settings.builder().put(updatedSettings).put(BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), "1024b"), + largeSnapshotPool && randomBoolean() ); assertSame(repoOnMaster, getRepositoryOnNode(repoName, masterNode)); assertSame(repoOnDataNode, getRepositoryOnNode(repoName, dataNode)); @@ -84,16 +86,18 @@ largeSnapshotPool && randomBoolean() // this setting update will fail so we can set the verification parameter randomly even if the SNAPSHOT pool is already blocked // since we will never actually get to the verification step createRepository( - repoName, - "mock", - Settings.builder().put(repoOnMaster.getMetadata().settings()).put("location", randomRepoPath()), - randomBoolean() + repoName, + "mock", + Settings.builder().put(repoOnMaster.getMetadata().settings()).put("location", randomRepoPath()), + randomBoolean() ); } catch (Exception e) { final Throwable ise = ExceptionsHelper.unwrap(e, IllegalStateException.class); assertThat(ise, instanceOf(IllegalStateException.class)); - assertEquals(ise.getMessage(), - "trying to modify or unregister repository [test-repo] that is currently used (snapshot is in progress)"); + assertEquals( + ise.getMessage(), + "trying to modify or unregister repository [test-repo] that is currently used (snapshot is in progress)" + ); } logger.info("--> verify that we can update [{}] dynamically", MockRepository.DUMMY_UPDATABLE_SETTING_NAME); @@ -101,11 +105,12 @@ largeSnapshotPool && randomBoolean() // we only run concurrent verification when we have a large SNAPSHOT pool on the data node because otherwise the verification would // deadlock since the small pool is already blocked by the snapshot on the data node createRepository( - repoName, - "mock", - Settings.builder().put(repoOnMaster.getMetadata().settings()) - .put(MockRepository.DUMMY_UPDATABLE_SETTING_NAME, dummySettingValue), - largeSnapshotPool && randomBoolean() + repoName, + "mock", + Settings.builder() + .put(repoOnMaster.getMetadata().settings()) + .put(MockRepository.DUMMY_UPDATABLE_SETTING_NAME, dummySettingValue), + largeSnapshotPool && randomBoolean() ); final Repository newRepoOnMaster = getRepositoryOnNode(repoName, masterNode); assertSame(repoOnMaster, newRepoOnMaster); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java index 3ad9ed641b2a2..eb9f60cebce47 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryCleanupIT.java @@ -41,8 +41,9 @@ public void testMasterFailoverDuringCleanup() throws Exception { ensureStableCluster(nodeCount - 1); logger.info("--> wait for cleanup to finish and disappear from cluster state"); - awaitClusterState(state -> - state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false); + awaitClusterState( + state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false + ); try { cleanupFuture.get(); @@ -62,16 +63,21 @@ public void testRepeatCleanupsDontRemove() throws Exception { assertFutureThrows(client().admin().cluster().prepareCleanupRepository("test-repo").execute(), IllegalStateException.class); logger.info("--> ensure cleanup is still in progress"); - final RepositoryCleanupInProgress cleanup = - client().admin().cluster().prepareState().get().getState().custom(RepositoryCleanupInProgress.TYPE); + final RepositoryCleanupInProgress cleanup = client().admin() + .cluster() + .prepareState() + .get() + .getState() + .custom(RepositoryCleanupInProgress.TYPE); assertTrue(cleanup.hasCleanupInProgress()); logger.info("--> unblocking master node"); unblockNode("test-repo", internalCluster().getMasterName()); logger.info("--> wait for cleanup to finish and disappear from cluster state"); - awaitClusterState(state -> - state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false); + awaitClusterState( + state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() == false + ); final ExecutionException e = expectThrows(ExecutionException.class, cleanupFuture::get); final Throwable ioe = ExceptionsHelper.unwrap(e, IOException.class); @@ -87,15 +93,22 @@ private ActionFuture startBlockedCleanup(String repoN createRepository(repoName, "mock"); logger.info("--> snapshot"); - client().admin().cluster().prepareCreateSnapshot(repoName, "test-snap") - .setWaitForCompletion(true).get(); + client().admin().cluster().prepareCreateSnapshot(repoName, "test-snap").setWaitForCompletion(true).get(); final BlobStoreRepository repository = getRepositoryOnMaster(repoName); logger.info("--> creating a garbage data blob"); final PlainActionFuture garbageFuture = PlainActionFuture.newFuture(); - repository.threadPool().generic().execute(ActionRunnable.run(garbageFuture, () -> repository.blobStore() - .blobContainer(repository.basePath()).writeBlob("snap-foo.dat", new BytesArray(new byte[1]), true))); + repository.threadPool() + .generic() + .execute( + ActionRunnable.run( + garbageFuture, + () -> repository.blobStore() + .blobContainer(repository.basePath()) + .writeBlob("snap-foo.dat", new BytesArray(new byte[1]), true) + ) + ); garbageFuture.get(); blockMasterFromFinalizingSnapshotOnIndexFile(repoName); @@ -103,13 +116,17 @@ private ActionFuture startBlockedCleanup(String repoN logger.info("--> starting repository cleanup"); // running from a non-master client because shutting down a master while a request to it is pending might result in the future // never completing - final ActionFuture future = - internalCluster().nonMasterClient().admin().cluster().prepareCleanupRepository(repoName).execute(); + final ActionFuture future = internalCluster().nonMasterClient() + .admin() + .cluster() + .prepareCleanupRepository(repoName) + .execute(); final String masterNode = internalCluster().getMasterName(); waitForBlock(masterNode, repoName); - awaitClusterState(state -> - state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()); + awaitClusterState( + state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() + ); return future; } @@ -121,8 +138,11 @@ public void testCleanupOldIndexN() throws ExecutionException, InterruptedExcepti logger.info("--> create three snapshots"); for (int i = 0; i < 3; ++i) { - CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(repoName, "test-snap-" + i) - .setWaitForCompletion(true).get(); + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, "test-snap-" + i) + .setWaitForCompletion(true) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().state(), is(SnapshotState.SUCCESS)); } @@ -131,9 +151,16 @@ public void testCleanupOldIndexN() throws ExecutionException, InterruptedExcepti for (int i = 0; i < 2; ++i) { final PlainActionFuture createOldIndexNFuture = PlainActionFuture.newFuture(); final int generation = i; - repository.threadPool().generic().execute(ActionRunnable.run(createOldIndexNFuture, () -> repository.blobStore() - .blobContainer(repository.basePath()).writeBlob(BlobStoreRepository.INDEX_FILE_PREFIX + generation, - new BytesArray(new byte[1]), true))); + repository.threadPool() + .generic() + .execute( + ActionRunnable.run( + createOldIndexNFuture, + () -> repository.blobStore() + .blobContainer(repository.basePath()) + .writeBlob(BlobStoreRepository.INDEX_FILE_PREFIX + generation, new BytesArray(new byte[1]), true) + ) + ); createOldIndexNFuture.get(); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreSizeLimitIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreSizeLimitIT.java index d3435eb1cece7..73ad2737f1f10 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreSizeLimitIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreSizeLimitIT.java @@ -25,14 +25,24 @@ public class BlobStoreSizeLimitIT extends AbstractSnapshotIntegTestCase { public void testBlobStoreSizeIsLimited() throws Exception { final String repoName = "test-repo"; final int maxSnapshots = randomIntBetween(1, 10); - createRepository(repoName, FsRepository.TYPE, Settings.builder() - .put(BlobStoreRepository.MAX_SNAPSHOTS_SETTING.getKey(), maxSnapshots).put("location", randomRepoPath())); + createRepository( + repoName, + FsRepository.TYPE, + Settings.builder().put(BlobStoreRepository.MAX_SNAPSHOTS_SETTING.getKey(), maxSnapshots).put("location", randomRepoPath()) + ); final List snapshotNames = createNSnapshots(repoName, maxSnapshots); final ActionFuture failingSnapshotFuture = startFullSnapshot(repoName, "failing-snapshot"); final RepositoryException repositoryException = expectThrows(RepositoryException.class, failingSnapshotFuture::actionGet); - assertThat(repositoryException.getMessage(), Matchers.endsWith( - "Cannot add another snapshot to this repository as it already contains [" + maxSnapshots + - "] snapshots and is configured to hold up to [" + maxSnapshots + "] snapshots only.")); + assertThat( + repositoryException.getMessage(), + Matchers.endsWith( + "Cannot add another snapshot to this repository as it already contains [" + + maxSnapshots + + "] snapshots and is configured to hold up to [" + + maxSnapshots + + "] snapshots only." + ) + ); assertEquals(repositoryException.repository(), repoName); assertAcked(startDeleteSnapshot(repoName, randomFrom(snapshotNames)).get()); createFullSnapshot(repoName, "last-snapshot"); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIntegTests.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIntegTests.java index efc395652af08..62ea73c214c0e 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIntegTests.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIntegTests.java @@ -17,9 +17,7 @@ public class FsBlobStoreRepositoryIntegTests extends ESFsBasedRepositoryIntegTes @Override protected Settings repositorySettings(String repositoryName) { - final Settings.Builder settings = Settings.builder() - .put("compress", randomBoolean()) - .put("location", randomRepoPath()); + final Settings.Builder settings = Settings.builder().put("compress", randomBoolean()).put("location", randomRepoPath()); if (randomBoolean()) { long size = 1 << randomInt(10); settings.put("chunk_size", new ByteSizeValue(size, ByteSizeUnit.KB)); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedRestoreIT.java index a80f4a8ea6d6f..50fdda6ae7344 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/AbortedRestoreIT.java @@ -52,14 +52,20 @@ public void testAbortedRestoreAlsoAbortFileRestores() throws Exception { failReadsAllDataNodes(repositoryName); logger.info("--> starting restore"); - final ActionFuture future = client().admin().cluster().prepareRestoreSnapshot(repositoryName, snapshotName) + final ActionFuture future = client().admin() + .cluster() + .prepareRestoreSnapshot(repositoryName, snapshotName) .setWaitForCompletion(true) .setIndices(indexName) .execute(); assertBusy(() -> { - final RecoveryResponse recoveries = client().admin().indices().prepareRecoveries(indexName) - .setIndicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN).setActiveOnly(true).get(); + final RecoveryResponse recoveries = client().admin() + .indices() + .prepareRecoveries(indexName) + .setIndicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN) + .setActiveOnly(true) + .get(); assertThat(recoveries.hasRecoveries(), is(true)); final List shardRecoveries = recoveries.shardRecoveryStates().get(indexName); assertThat(shardRecoveries, hasSize(1)); @@ -71,8 +77,8 @@ public void testAbortedRestoreAlsoAbortFileRestores() throws Exception { } }); - final ThreadPool.Info snapshotThreadPoolInfo = - internalCluster().getInstance(ThreadPool.class, dataNode).info(ThreadPool.Names.SNAPSHOT); + final ThreadPool.Info snapshotThreadPoolInfo = internalCluster().getInstance(ThreadPool.class, dataNode) + .info(ThreadPool.Names.SNAPSHOT); assertThat(snapshotThreadPoolInfo.getMax(), greaterThan(0)); logger.info("--> waiting for snapshot thread [max={}] pool to be full", snapshotThreadPoolInfo.getMax()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/BlobStoreIncrementalityIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/BlobStoreIncrementalityIT.java index 731b724a38a5b..af9dd8473103f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/BlobStoreIncrementalityIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/BlobStoreIncrementalityIT.java @@ -38,10 +38,14 @@ public void testIncrementalBehaviorOnPrimaryFailover() throws InterruptedExcepti internalCluster().startMasterOnlyNode(); final String primaryNode = internalCluster().startDataOnlyNode(); final String indexName = "test-index"; - createIndex(indexName, Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0).build()); + createIndex( + indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0) + .build() + ); ensureYellow(indexName); final String newPrimary = internalCluster().startDataOnlyNode(); final Collection toDelete = new ArrayList<>(); @@ -145,8 +149,12 @@ public void testForceMergeCausesFullSnapshot() throws Exception { client().admin().cluster().prepareCreateSnapshot(repo, snapshot1).setIndices(indexName).setWaitForCompletion(true).get(); logger.info("--> force merging down to a single segment"); - final ForceMergeResponse forceMergeResponse = - client().admin().indices().prepareForceMerge(indexName).setMaxNumSegments(1).setFlush(true).get(); + final ForceMergeResponse forceMergeResponse = client().admin() + .indices() + .prepareForceMerge(indexName) + .setMaxNumSegments(1) + .setFlush(true) + .get(); assertThat(forceMergeResponse.getFailedShards(), is(0)); final String snapshot2 = "snap-2"; @@ -154,8 +162,7 @@ public void testForceMergeCausesFullSnapshot() throws Exception { client().admin().cluster().prepareCreateSnapshot(repo, snapshot2).setIndices(indexName).setWaitForCompletion(true).get(); logger.info("--> asserting that the two snapshots refer to different files in the repository"); - final SnapshotStats secondSnapshotShardStatus = - getStats(repo, snapshot2).getIndices().get(indexName).getShards().get(0).getStats(); + final SnapshotStats secondSnapshotShardStatus = getStats(repo, snapshot2).getIndices().get(indexName).getShards().get(0).getStats(); assertThat(secondSnapshotShardStatus.getIncrementalFileCount(), greaterThan(0)); } @@ -167,13 +174,11 @@ private void assertCountInIndexThenDelete(String index, long expectedCount) { } private void assertTwoIdenticalShardSnapshots(String repo, String indexName, String snapshot1, String snapshot2) { - logger.info( - "--> asserting that snapshots [{}] and [{}] are referring to the same files in the repository", snapshot1, snapshot2); + logger.info("--> asserting that snapshots [{}] and [{}] are referring to the same files in the repository", snapshot1, snapshot2); final SnapshotStats firstSnapshotShardStatus = getStats(repo, snapshot1).getIndices().get(indexName).getShards().get(0).getStats(); final int totalFilesInShard = firstSnapshotShardStatus.getTotalFileCount(); assertThat(totalFilesInShard, greaterThan(0)); - final SnapshotStats secondSnapshotShardStatus = - getStats(repo, snapshot2).getIndices().get(indexName).getShards().get(0).getStats(); + final SnapshotStats secondSnapshotShardStatus = getStats(repo, snapshot2).getIndices().get(indexName).getShards().get(0).getStats(); assertThat(secondSnapshotShardStatus.getTotalFileCount(), is(totalFilesInShard)); assertThat(secondSnapshotShardStatus.getIncrementalFileCount(), is(0)); } @@ -184,8 +189,14 @@ private SnapshotStatus getStats(String repository, String snapshot) { private void ensureRestoreSingleShardSuccessfully(String repo, String indexName, String snapshot, String indexSuffix) { logger.info("--> restoring [{}]", snapshot); - final RestoreSnapshotResponse restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot(repo, snapshot) - .setIndices(indexName).setRenamePattern("(.+)").setRenameReplacement("$1" + indexSuffix).setWaitForCompletion(true).get(); + final RestoreSnapshotResponse restoreSnapshotResponse = client().admin() + .cluster() + .prepareRestoreSnapshot(repo, snapshot) + .setIndices(indexName) + .setRenamePattern("(.+)") + .setRenameReplacement("$1" + indexSuffix) + .setWaitForCompletion(true) + .get(); final RestoreInfo restoreInfo = restoreSnapshotResponse.getRestoreInfo(); assertThat(restoreInfo.totalShards(), is(1)); assertThat(restoreInfo.failedShards(), is(0)); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java index e2d9617d388eb..9ca7ca4e6d0dc 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java @@ -82,8 +82,9 @@ public void testShardClone() throws Exception { } else { currentShardGen = repositoryData.shardGenerations().getShardGen(indexId, shardId); } - final ShardSnapshotResult shardSnapshotResult = PlainActionFuture.get(f -> repository.cloneShardSnapshot( - sourceSnapshotInfo.snapshotId(), targetSnapshotId, repositoryShardId, currentShardGen, f)); + final ShardSnapshotResult shardSnapshotResult = PlainActionFuture.get( + f -> repository.cloneShardSnapshot(sourceSnapshotInfo.snapshotId(), targetSnapshotId, repositoryShardId, currentShardGen, f) + ); final String newShardGeneration = shardSnapshotResult.getGeneration(); if (useBwCFormat) { @@ -92,8 +93,11 @@ public void testShardClone() throws Exception { } final BlobStoreIndexShardSnapshot targetShardSnapshot = readShardSnapshot(repository, repositoryShardId, targetSnapshotId); - final BlobStoreIndexShardSnapshot sourceShardSnapshot = - readShardSnapshot(repository, repositoryShardId, sourceSnapshotInfo.snapshotId()); + final BlobStoreIndexShardSnapshot sourceShardSnapshot = readShardSnapshot( + repository, + repositoryShardId, + sourceSnapshotInfo.snapshotId() + ); assertThat(targetShardSnapshot.incrementalFileCount(), is(0)); final List sourceFiles = sourceShardSnapshot.indexFiles(); final List targetFiles = targetShardSnapshot.indexFiles(); @@ -108,8 +112,9 @@ public void testShardClone() throws Exception { assertTrue(snapshotFiles.get(0).isSame(snapshotFiles.get(1))); // verify that repeated cloning is idempotent - final ShardSnapshotResult shardSnapshotResult2 = PlainActionFuture.get(f -> repository.cloneShardSnapshot( - sourceSnapshotInfo.snapshotId(), targetSnapshotId, repositoryShardId, newShardGeneration, f)); + final ShardSnapshotResult shardSnapshotResult2 = PlainActionFuture.get( + f -> repository.cloneShardSnapshot(sourceSnapshotInfo.snapshotId(), targetSnapshotId, repositoryShardId, newShardGeneration, f) + ); assertEquals(newShardGeneration, shardSnapshotResult2.getGeneration()); assertEquals(shardSnapshotResult.getSegmentCount(), shardSnapshotResult2.getSegmentCount()); assertEquals(shardSnapshotResult.getSize(), shardSnapshotResult2.getSize()); @@ -134,7 +139,9 @@ public void testCloneSnapshotIndex() throws Exception { assertAcked(startClone(repoName, sourceSnapshot, targetSnapshot, indexName).get()); final List status = clusterAdmin().prepareSnapshotStatus(repoName) - .setSnapshots(sourceSnapshot, targetSnapshot).get().getSnapshots(); + .setSnapshots(sourceSnapshot, targetSnapshot) + .get() + .getSnapshots(); assertThat(status, hasSize(2)); final SnapshotIndexStatus status1 = status.get(0).getIndices().get(indexName); final SnapshotIndexStatus status2 = status.get(1).getIndices().get(indexName); @@ -161,14 +168,18 @@ public void testClonePreventsSnapshotDelete() throws Exception { waitForBlock(masterName, repoName); assertFalse(cloneFuture.isDone()); - ConcurrentSnapshotExecutionException ex = expectThrows(ConcurrentSnapshotExecutionException.class, - () -> startDeleteSnapshot(repoName, sourceSnapshot).actionGet()); + ConcurrentSnapshotExecutionException ex = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> startDeleteSnapshot(repoName, sourceSnapshot).actionGet() + ); assertThat(ex.getMessage(), containsString("cannot delete snapshot while it is being cloned")); unblockNode(repoName, masterName); assertAcked(cloneFuture.get()); final List status = clusterAdmin().prepareSnapshotStatus(repoName) - .setSnapshots(sourceSnapshot, targetSnapshot).get().getSnapshots(); + .setSnapshots(sourceSnapshot, targetSnapshot) + .get() + .getSnapshots(); assertThat(status, hasSize(2)); final SnapshotIndexStatus status1 = status.get(0).getIndices().get(indexName); final SnapshotIndexStatus status2 = status.get(1).getIndices().get(indexName); @@ -190,8 +201,7 @@ public void testConcurrentCloneAndSnapshot() throws Exception { indexRandomDocs(indexName, randomIntBetween(20, 100)); final String targetSnapshot = "target-snapshot"; - final ActionFuture snapshot2Future = - startFullSnapshotBlockedOnDataNode("snapshot-2", repoName, dataNode); + final ActionFuture snapshot2Future = startFullSnapshotBlockedOnDataNode("snapshot-2", repoName, dataNode); waitForBlock(dataNode, repoName); final ActionFuture cloneFuture = startClone(repoName, sourceSnapshot, targetSnapshot, indexName); awaitNumberOfSnapshotsInProgress(2); @@ -220,8 +230,9 @@ public void testLongRunningCloneAllowsConcurrentSnapshot() throws Exception { final String indexFast = "index-fast"; createIndexWithRandomDocs(indexFast, randomIntBetween(20, 100)); - assertSuccessful(clusterAdmin().prepareCreateSnapshot(repoName, "fast-snapshot") - .setIndices(indexFast).setWaitForCompletion(true).execute()); + assertSuccessful( + clusterAdmin().prepareCreateSnapshot(repoName, "fast-snapshot").setIndices(indexFast).setWaitForCompletion(true).execute() + ); assertThat(cloneFuture.isDone(), is(false)); unblockNode(repoName, masterNode); @@ -244,8 +255,10 @@ public void testLongRunningSnapshotAllowsConcurrentClone() throws Exception { createIndexWithRandomDocs(indexFast, randomIntBetween(20, 100)); blockDataNode(repoName, dataNode); - final ActionFuture snapshotFuture = clusterAdmin() - .prepareCreateSnapshot(repoName, "fast-snapshot").setIndices(indexFast).setWaitForCompletion(true).execute(); + final ActionFuture snapshotFuture = clusterAdmin().prepareCreateSnapshot(repoName, "fast-snapshot") + .setIndices(indexFast) + .setWaitForCompletion(true) + .execute(); waitForBlock(dataNode, repoName); final String targetSnapshot = "target-snapshot"; @@ -276,8 +289,10 @@ public void testDeletePreventsClone() throws Exception { waitForBlock(masterName, repoName); assertFalse(deleteFuture.isDone()); - ConcurrentSnapshotExecutionException ex = expectThrows(ConcurrentSnapshotExecutionException.class, () -> - startClone(repoName, sourceSnapshot, targetSnapshot, indexName).actionGet()); + ConcurrentSnapshotExecutionException ex = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> startClone(repoName, sourceSnapshot, targetSnapshot, indexName).actionGet() + ); assertThat(ex.getMessage(), containsString("cannot clone from snapshot that is being deleted")); unblockNode(repoName, masterName); @@ -357,8 +372,7 @@ public void testMasterFailoverDuringCloneStep1() throws Exception { blockMasterOnReadIndexMeta(repoName); final String cloneName = "target-snapshot"; - final ActionFuture cloneFuture = - startCloneFromDataNode(repoName, sourceSnapshot, cloneName, testIndex); + final ActionFuture cloneFuture = startCloneFromDataNode(repoName, sourceSnapshot, cloneName, testIndex); awaitNumberOfSnapshotsInProgress(1); final String masterNode = internalCluster().getMasterName(); waitForBlock(masterNode, repoName); @@ -375,8 +389,8 @@ public void testMasterFailoverDuringCloneStep1() throws Exception { awaitNoMoreRunningOperations(); // Check if the clone operation worked out by chance as a result of the clone request being retried because of the master failover - cloneSucceeded = cloneSucceeded || - getRepositoryData(repoName).getSnapshotIds().stream().anyMatch(snapshotId -> snapshotId.getName().equals(cloneName)); + cloneSucceeded = cloneSucceeded + || getRepositoryData(repoName).getSnapshotIds().stream().anyMatch(snapshotId -> snapshotId.getName().equals(cloneName)); assertAllSnapshotsSuccessful(getRepositoryData(repoName), cloneSucceeded ? 2 : 1); } @@ -392,8 +406,10 @@ public void testFailsOnCloneMissingIndices() { final String snapshotName = "snapshot"; createFullSnapshot(repoName, snapshotName); - expectThrows(IndexNotFoundException.class, - () -> startClone(repoName, snapshotName, "target-snapshot", "does-not-exist").actionGet()); + expectThrows( + IndexNotFoundException.class, + () -> startClone(repoName, snapshotName, "target-snapshot", "does-not-exist").actionGet() + ); } public void testMasterFailoverDuringCloneStep2() throws Exception { @@ -457,17 +473,28 @@ public void testDoesNotStartOnBrokenSourceSnapshot() throws Exception { final String sourceSnapshot = "source-snapshot"; blockDataNode(repoName, dataNode); final Client masterClient = internalCluster().masterClient(); - final ActionFuture sourceSnapshotFuture = masterClient.admin().cluster() - .prepareCreateSnapshot(repoName, sourceSnapshot).setWaitForCompletion(true).execute(); + final ActionFuture sourceSnapshotFuture = masterClient.admin() + .cluster() + .prepareCreateSnapshot(repoName, sourceSnapshot) + .setWaitForCompletion(true) + .execute(); awaitNumberOfSnapshotsInProgress(1); waitForBlock(dataNode, repoName); internalCluster().restartNode(dataNode); assertThat(sourceSnapshotFuture.get().getSnapshotInfo().state(), is(SnapshotState.PARTIAL)); - final SnapshotException sne = expectThrows(SnapshotException.class, () -> startClone(masterClient, repoName, sourceSnapshot, - "target-snapshot", testIndex).actionGet(TimeValue.timeValueSeconds(30L))); - assertThat(sne.getMessage(), containsString("Can't clone index [" + getRepositoryData(repoName).resolveIndexId(testIndex) + - "] because its snapshot was not successful.")); + final SnapshotException sne = expectThrows( + SnapshotException.class, + () -> startClone(masterClient, repoName, sourceSnapshot, "target-snapshot", testIndex).actionGet( + TimeValue.timeValueSeconds(30L) + ) + ); + assertThat( + sne.getMessage(), + containsString( + "Can't clone index [" + getRepositoryData(repoName).resolveIndexId(testIndex) + "] because its snapshot was not successful." + ) + ); } public void testStartSnapshotWithSuccessfulShardClonePendingFinalization() throws Exception { @@ -551,8 +578,8 @@ public void testStartCloneWithSuccessfulShardSnapshotPendingFinalization() throw logger.info("--> wait for clone to start fully with shards assigned in the cluster state"); try { awaitClusterState(clusterState -> { - final List entries = - clusterState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries(); + final List entries = clusterState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .entries(); return entries.size() == 2 && entries.get(1).clones().isEmpty() == false; }); assertFalse(blockedSnapshot.isDone()); @@ -598,7 +625,7 @@ public void testStartCloneDuringRunningDelete() throws Exception { return true; } } - return false; + return false; }); unblockNode(repoName, masterName); assertAcked(deleteFuture.get()); @@ -630,26 +657,43 @@ public void testManyConcurrentClonesStartOutOfOrder() throws Exception { final ActionFuture clone2 = startClone(repoName, sourceSnapshot, "target-snapshot-2", testIndex); awaitNumberOfSnapshotsInProgress(2); - awaitClusterState(state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) - .entries().stream().anyMatch(entry -> entry.state().completed())); + awaitClusterState( + state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .entries() + .stream() + .anyMatch(entry -> entry.state().completed()) + ); repo.unblock(); assertAcked(clone1.get()); assertAcked(clone2.get()); } - private ActionFuture startCloneFromDataNode(String repoName, String sourceSnapshot, String targetSnapshot, - String... indices) { + private ActionFuture startCloneFromDataNode( + String repoName, + String sourceSnapshot, + String targetSnapshot, + String... indices + ) { return startClone(dataNodeClient(), repoName, sourceSnapshot, targetSnapshot, indices); } - private ActionFuture startClone(String repoName, String sourceSnapshot, String targetSnapshot, - String... indices) { + private ActionFuture startClone( + String repoName, + String sourceSnapshot, + String targetSnapshot, + String... indices + ) { return startClone(client(), repoName, sourceSnapshot, targetSnapshot, indices); } - private static ActionFuture startClone(Client client, String repoName, String sourceSnapshot, - String targetSnapshot, String... indices) { + private static ActionFuture startClone( + Client client, + String repoName, + String sourceSnapshot, + String targetSnapshot, + String... indices + ) { return client.admin().cluster().prepareCloneSnapshot(repoName, sourceSnapshot, targetSnapshot).setIndices(indices).execute(); } @@ -672,17 +716,44 @@ private static void assertAllSnapshotsSuccessful(RepositoryData repositoryData, } } - private static BlobStoreIndexShardSnapshots readShardGeneration(BlobStoreRepository repository, RepositoryShardId repositoryShardId, - String generation) { - return PlainActionFuture.get(f -> repository.threadPool().generic().execute(ActionRunnable.supply(f, - () -> BlobStoreRepository.INDEX_SHARD_SNAPSHOTS_FORMAT.read(repository.shardContainer(repositoryShardId.index(), - repositoryShardId.shardId()), generation, NamedXContentRegistry.EMPTY)))); + private static BlobStoreIndexShardSnapshots readShardGeneration( + BlobStoreRepository repository, + RepositoryShardId repositoryShardId, + String generation + ) { + return PlainActionFuture.get( + f -> repository.threadPool() + .generic() + .execute( + ActionRunnable.supply( + f, + () -> BlobStoreRepository.INDEX_SHARD_SNAPSHOTS_FORMAT.read( + repository.shardContainer(repositoryShardId.index(), repositoryShardId.shardId()), + generation, + NamedXContentRegistry.EMPTY + ) + ) + ) + ); } - private static BlobStoreIndexShardSnapshot readShardSnapshot(BlobStoreRepository repository, RepositoryShardId repositoryShardId, - SnapshotId snapshotId) { - return PlainActionFuture.get(f -> repository.threadPool().generic().execute(ActionRunnable.supply(f, - () -> repository.loadShardSnapshot(repository.shardContainer(repositoryShardId.index(), repositoryShardId.shardId()), - snapshotId)))); + private static BlobStoreIndexShardSnapshot readShardSnapshot( + BlobStoreRepository repository, + RepositoryShardId repositoryShardId, + SnapshotId snapshotId + ) { + return PlainActionFuture.get( + f -> repository.threadPool() + .generic() + .execute( + ActionRunnable.supply( + f, + () -> repository.loadShardSnapshot( + repository.shardContainer(repositoryShardId.index(), repositoryShardId.shardId()), + snapshotId + ) + ) + ) + ); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java index 93cf8ae59ebe7..24755fa163e73 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java @@ -72,9 +72,10 @@ protected Collection> nodePlugins() { @Override protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings)) - .put(AbstractDisruptionTestCase.DEFAULT_SETTINGS) - .build(); + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(AbstractDisruptionTestCase.DEFAULT_SETTINGS) + .build(); } public void testLongRunningSnapshotAllowsConcurrentSnapshot() throws Exception { @@ -84,16 +85,25 @@ public void testLongRunningSnapshotAllowsConcurrentSnapshot() throws Exception { createRepository(repoName, "mock"); createIndexWithContent("index-slow"); - final ActionFuture createSlowFuture = - startFullSnapshotBlockedOnDataNode("slow-snapshot", repoName, dataNode); + final ActionFuture createSlowFuture = startFullSnapshotBlockedOnDataNode( + "slow-snapshot", + repoName, + dataNode + ); final String dataNode2 = internalCluster().startDataOnlyNode(); ensureStableCluster(3); final String indexFast = "index-fast"; createIndexWithContent(indexFast, dataNode2, dataNode); - assertSuccessful(client().admin().cluster().prepareCreateSnapshot(repoName, "fast-snapshot") - .setIndices(indexFast).setWaitForCompletion(true).execute()); + assertSuccessful( + client().admin() + .cluster() + .prepareCreateSnapshot(repoName, "fast-snapshot") + .setIndices(indexFast) + .setWaitForCompletion(true) + .execute() + ); assertThat(createSlowFuture.isDone(), is(false)); unblockNode(repoName, dataNode); @@ -115,8 +125,11 @@ public void testDeletesAreBatched() throws Exception { createIndexWithContent("index-slow"); - final ActionFuture createSlowFuture = - startFullSnapshotBlockedOnDataNode("blocked-snapshot", repoName, dataNode); + final ActionFuture createSlowFuture = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot", + repoName, + dataNode + ); final Collection> deleteFutures = new ArrayList<>(); while (snapshotNames.isEmpty() == false) { @@ -164,12 +177,14 @@ public void testBlockedRepoDoesNotBlockOtherRepos() throws Exception { ensureGreen(); createIndexWithContent("index-slow"); - final ActionFuture createSlowFuture = - startAndBlockFailingFullSnapshot(blockedRepoName, "blocked-snapshot"); + final ActionFuture createSlowFuture = startAndBlockFailingFullSnapshot(blockedRepoName, "blocked-snapshot"); - client().admin().cluster().prepareCreateSnapshot(otherRepoName, "snapshot") - .setIndices("does-not-exist-*") - .setWaitForCompletion(false).get(); + client().admin() + .cluster() + .prepareCreateSnapshot(otherRepoName, "snapshot") + .setIndices("does-not-exist-*") + .setWaitForCompletion(false) + .get(); unblockNode(blockedRepoName, internalCluster().getMasterName()); expectThrows(SnapshotException.class, createSlowFuture::actionGet); @@ -188,8 +203,11 @@ public void testMultipleReposAreIndependent() throws Exception { createRepository(otherRepoName, "fs"); createIndexWithContent("test-index"); - final ActionFuture createSlowFuture = - startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + final ActionFuture createSlowFuture = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot", + blockedRepoName, + dataNode + ); logger.info("--> waiting for concurrent snapshot(s) to finish"); createNSnapshots(otherRepoName, randomIntBetween(1, 5)); @@ -209,8 +227,11 @@ public void testMultipleReposAreIndependent2() throws Exception { createRepository(otherRepoName, "fs"); createIndexWithContent("test-index"); - final ActionFuture createSlowFuture = - startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + final ActionFuture createSlowFuture = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot", + blockedRepoName, + dataNode + ); logger.info("--> waiting for concurrent snapshot(s) to finish"); createNSnapshots(otherRepoName, randomIntBetween(1, 5)); @@ -229,7 +250,7 @@ public void testMultipleReposAreIndependent3() throws Exception { createRepository(otherRepoName, "fs"); createIndexWithContent("test-index"); - createFullSnapshot( blockedRepoName, "blocked-snapshot"); + createFullSnapshot(blockedRepoName, "blocked-snapshot"); blockNodeOnAnyFiles(blockedRepoName, masterNode); final ActionFuture slowDeleteFuture = startDeleteSnapshot(blockedRepoName, "*"); @@ -275,8 +296,11 @@ public void testAbortOneOfMultipleSnapshots() throws Exception { createIndexWithContent(firstIndex); final String firstSnapshot = "snapshot-one"; - final ActionFuture firstSnapshotResponse = - startFullSnapshotBlockedOnDataNode(firstSnapshot, repoName, dataNode); + final ActionFuture firstSnapshotResponse = startFullSnapshotBlockedOnDataNode( + firstSnapshot, + repoName, + dataNode + ); final String dataNode2 = internalCluster().startDataOnlyNode(); ensureStableCluster(3); @@ -296,8 +320,12 @@ public void testAbortOneOfMultipleSnapshots() throws Exception { awaitNDeletionsInProgress(1); logger.info("--> start third snapshot"); - final ActionFuture thirdSnapshotResponse = client().admin().cluster() - .prepareCreateSnapshot(repoName, "snapshot-three").setIndices(secondIndex).setWaitForCompletion(true).execute(); + final ActionFuture thirdSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, "snapshot-three") + .setIndices(secondIndex) + .setWaitForCompletion(true) + .execute(); assertThat(firstSnapshotResponse.isDone(), is(false)); assertThat(secondSnapshotResponse.isDone(), is(false)); @@ -313,8 +341,10 @@ public void testAbortOneOfMultipleSnapshots() throws Exception { assertThat(deleteSnapshotsResponse.get().isAcknowledged(), is(true)); logger.info("--> verify that the first snapshot is gone"); - assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), - containsInAnyOrder(secondSnapshotInfo, thirdSnapshotInfo)); + assertThat( + client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), + containsInAnyOrder(secondSnapshotInfo, thirdSnapshotInfo) + ); } public void testCascadedAborts() throws Exception { @@ -325,8 +355,11 @@ public void testCascadedAborts() throws Exception { createIndexWithContent("index-one"); final String firstSnapshot = "snapshot-one"; - final ActionFuture firstSnapshotResponse = - startFullSnapshotBlockedOnDataNode(firstSnapshot, repoName, dataNode); + final ActionFuture firstSnapshotResponse = startFullSnapshotBlockedOnDataNode( + firstSnapshot, + repoName, + dataNode + ); final String dataNode2 = internalCluster().startDataOnlyNode(); ensureStableCluster(3); @@ -427,8 +460,9 @@ public void testMasterFailOverWithQueuedDeletes() throws Exception { logger.info("--> waiting for second snapshot to finish and the other two snapshots to become aborted"); assertBusy(() -> { assertThat(currentSnapshots(repoName), hasSize(2)); - for (SnapshotsInProgress.Entry entry - : clusterService().state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) { + for (SnapshotsInProgress.Entry entry : clusterService().state() + .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .entries()) { assertThat(entry.state(), is(SnapshotsInProgress.State.ABORTED)); assertThat(entry.snapshot().getSnapshotId().getName(), not(secondSnapshot)); } @@ -586,16 +620,23 @@ public void testQueuedOperationsOnMasterDisconnect() throws Exception { internalCluster().setDisruptionScheme(networkDisruption); blockNodeOnAnyFiles(repoName, masterNode); - ActionFuture firstDeleteFuture = client(masterNode).admin().cluster() - .prepareDeleteSnapshot(repoName, "*").execute(); + ActionFuture firstDeleteFuture = client(masterNode).admin() + .cluster() + .prepareDeleteSnapshot(repoName, "*") + .execute(); waitForBlock(masterNode, repoName); - final ActionFuture createThirdSnapshot = client(masterNode).admin().cluster() - .prepareCreateSnapshot(repoName, "snapshot-three").setWaitForCompletion(true).execute(); + final ActionFuture createThirdSnapshot = client(masterNode).admin() + .cluster() + .prepareCreateSnapshot(repoName, "snapshot-three") + .setWaitForCompletion(true) + .execute(); awaitNumberOfSnapshotsInProgress(1); - final ActionFuture secondDeleteFuture = - client(masterNode).admin().cluster().prepareDeleteSnapshot(repoName, "*").execute(); + final ActionFuture secondDeleteFuture = client(masterNode).admin() + .cluster() + .prepareDeleteSnapshot(repoName, "*") + .execute(); awaitNDeletionsInProgress(2); networkDisruption.startDisrupting(); @@ -624,15 +665,21 @@ public void testQueuedOperationsOnMasterDisconnectAndRepoFailure() throws Except internalCluster().setDisruptionScheme(networkDisruption); blockMasterFromFinalizingSnapshotOnIndexFile(repoName); - final ActionFuture firstFailedSnapshotFuture = - startFullSnapshotFromMasterClient(repoName, "failing-snapshot-1"); + final ActionFuture firstFailedSnapshotFuture = startFullSnapshotFromMasterClient( + repoName, + "failing-snapshot-1" + ); waitForBlock(masterNode, repoName); - final ActionFuture secondFailedSnapshotFuture = - startFullSnapshotFromMasterClient(repoName, "failing-snapshot-2"); + final ActionFuture secondFailedSnapshotFuture = startFullSnapshotFromMasterClient( + repoName, + "failing-snapshot-2" + ); awaitNumberOfSnapshotsInProgress(2); - final ActionFuture failedDeleteFuture = - client(masterNode).admin().cluster().prepareDeleteSnapshot(repoName, "*").execute(); + final ActionFuture failedDeleteFuture = client(masterNode).admin() + .cluster() + .prepareDeleteSnapshot(repoName, "*") + .execute(); awaitNDeletionsInProgress(1); networkDisruption.startDisrupting(); @@ -758,8 +805,10 @@ public void testQueuedSnapshotOperationsAndBrokenRepoOnMasterFailOverMultipleRep final ActionFuture deleteFuture = startDeleteFromNonMasterClient(blockedRepoName, "*"); waitForBlock(masterNode, blockedRepoName); awaitNDeletionsInProgress(1); - final ActionFuture createBlockedSnapshot = - startFullSnapshotFromNonMasterClient(blockedRepoName, "queued-snapshot"); + final ActionFuture createBlockedSnapshot = startFullSnapshotFromNonMasterClient( + blockedRepoName, + "queued-snapshot" + ); awaitNumberOfSnapshotsInProgress(1); final long generation = getRepositoryData(repoName).getGenId(); @@ -851,17 +900,30 @@ public void testQueuedSnapshotsWaitingForShardReady() throws Exception { allowNodes(testIndex, 1); logger.info("--> wait for relocations to start"); - assertBusy(() -> assertThat( - client().admin().cluster().prepareHealth(testIndex).execute().actionGet().getRelocatingShards(), greaterThan(0)), - 1L, TimeUnit.MINUTES); + assertBusy( + () -> assertThat( + client().admin().cluster().prepareHealth(testIndex).execute().actionGet().getRelocatingShards(), + greaterThan(0) + ), + 1L, + TimeUnit.MINUTES + ); logger.info("--> start two snapshots"); final String snapshotOne = "snap-1"; final String snapshotTwo = "snap-2"; - final ActionFuture snapOneResponse = client().admin().cluster() - .prepareCreateSnapshot(repoName, snapshotOne).setWaitForCompletion(false).setIndices(testIndex).execute(); - final ActionFuture snapTwoResponse = client().admin().cluster() - .prepareCreateSnapshot(repoName, snapshotTwo).setWaitForCompletion(false).setIndices(testIndex).execute(); + final ActionFuture snapOneResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotOne) + .setWaitForCompletion(false) + .setIndices(testIndex) + .execute(); + final ActionFuture snapTwoResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotTwo) + .setWaitForCompletion(false) + .setIndices(testIndex) + .execute(); snapOneResponse.get(); snapTwoResponse.get(); @@ -1009,12 +1071,21 @@ public void testStatusMultipleSnapshotsMultipleRepos() throws Exception { createRepository(otherBlockedRepoName, "mock"); createIndexWithContent("test-index"); - final ActionFuture createSlowFuture1 = - startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); - final ActionFuture createSlowFuture2 = - startFullSnapshotBlockedOnDataNode("blocked-snapshot-2", blockedRepoName, dataNode); - final ActionFuture createSlowFuture3 = - startFullSnapshotBlockedOnDataNode("other-blocked-snapshot", otherBlockedRepoName, dataNode); + final ActionFuture createSlowFuture1 = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot", + blockedRepoName, + dataNode + ); + final ActionFuture createSlowFuture2 = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot-2", + blockedRepoName, + dataNode + ); + final ActionFuture createSlowFuture3 = startFullSnapshotBlockedOnDataNode( + "other-blocked-snapshot", + otherBlockedRepoName, + dataNode + ); awaitNumberOfSnapshotsInProgress(3); assertSnapshotStatusCountOnRepo("_all", 3); @@ -1044,12 +1115,21 @@ public void testInterleavedAcrossMultipleRepos() throws Exception { createRepository(otherBlockedRepoName, "mock"); createIndexWithContent("test-index"); - final ActionFuture createSlowFuture1 = - startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); - final ActionFuture createSlowFuture2 = - startFullSnapshotBlockedOnDataNode("blocked-snapshot-2", blockedRepoName, dataNode); - final ActionFuture createSlowFuture3 = - startFullSnapshotBlockedOnDataNode("other-blocked-snapshot", otherBlockedRepoName, dataNode); + final ActionFuture createSlowFuture1 = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot", + blockedRepoName, + dataNode + ); + final ActionFuture createSlowFuture2 = startFullSnapshotBlockedOnDataNode( + "blocked-snapshot-2", + blockedRepoName, + dataNode + ); + final ActionFuture createSlowFuture3 = startFullSnapshotBlockedOnDataNode( + "other-blocked-snapshot", + otherBlockedRepoName, + dataNode + ); awaitNumberOfSnapshotsInProgress(3); unblockNode(blockedRepoName, dataNode); unblockNode(otherBlockedRepoName, dataNode); @@ -1107,8 +1187,15 @@ public void testConcurrentOperationsLimit() throws Exception { final int limitToTest = randomIntBetween(1, 3); final List snapshotNames = createNSnapshots(repoName, limitToTest + 1); - assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder().put( - SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.getKey(), limitToTest).build()).get()); + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings( + Settings.builder().put(SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.getKey(), limitToTest).build() + ) + .get() + ); blockNodeOnAnyFiles(repoName, masterName); int blockedSnapshots = 0; @@ -1128,10 +1215,20 @@ public void testConcurrentOperationsLimit() throws Exception { } waitForBlock(masterName, repoName); - final ConcurrentSnapshotExecutionException cse = expectThrows(ConcurrentSnapshotExecutionException.class, - () -> client().admin().cluster().prepareCreateSnapshot(repoName, "expected-to-fail").execute().actionGet()); - assertThat(cse.getMessage(), containsString("Cannot start another operation, already running [" + limitToTest + - "] operations and the current limit for concurrent snapshot operations is set to [" + limitToTest + "]")); + final ConcurrentSnapshotExecutionException cse = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> client().admin().cluster().prepareCreateSnapshot(repoName, "expected-to-fail").execute().actionGet() + ); + assertThat( + cse.getMessage(), + containsString( + "Cannot start another operation, already running [" + + limitToTest + + "] operations and the current limit for concurrent snapshot operations is set to [" + + limitToTest + + "]" + ) + ); boolean deleteAndAbortAll = false; if (deleteFuture == null && randomBoolean()) { deleteFuture = client().admin().cluster().prepareDeleteSnapshot(repoName, "*").execute(); @@ -1166,14 +1263,20 @@ public void testConcurrentSnapshotWorksWithOldVersionRepo() throws Exception { final String dataNode = internalCluster().startDataOnlyNode(); final String repoName = "test-repo"; final Path repoPath = randomRepoPath(); - createRepository(repoName, "mock", Settings.builder().put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) - .put("location", repoPath)); + createRepository( + repoName, + "mock", + Settings.builder().put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false).put("location", repoPath) + ); initWithSnapshotVersion(repoName, repoPath, SnapshotsService.OLD_SNAPSHOT_FORMAT); createIndexWithContent("index-slow"); - final ActionFuture createSlowFuture = - startFullSnapshotBlockedOnDataNode("slow-snapshot", repoName, dataNode); + final ActionFuture createSlowFuture = startFullSnapshotBlockedOnDataNode( + "slow-snapshot", + repoName, + dataNode + ); final String dataNode2 = internalCluster().startDataOnlyNode(); ensureStableCluster(3); @@ -1215,8 +1318,11 @@ public void testAbortNotStartedSnapshotWithoutIO() throws Exception { createRepository(repoName, "mock"); createIndexWithContent("test-index"); - final ActionFuture createSnapshot1Future = - startFullSnapshotBlockedOnDataNode("first-snapshot", repoName, dataNode); + final ActionFuture createSnapshot1Future = startFullSnapshotBlockedOnDataNode( + "first-snapshot", + repoName, + dataNode + ); final String snapshotTwo = "second-snapshot"; final ActionFuture createSnapshot2Future = startFullSnapshot(repoName, snapshotTwo); @@ -1260,8 +1366,10 @@ public void testStartWithSuccessfulShardSnapshotPendingFinalization() throws Exc } private static void assertSnapshotStatusCountOnRepo(String otherBlockedRepoName, int count) { - final SnapshotsStatusResponse snapshotsStatusResponse = - client().admin().cluster().prepareSnapshotStatus(otherBlockedRepoName).get(); + final SnapshotsStatusResponse snapshotsStatusResponse = client().admin() + .cluster() + .prepareSnapshotStatus(otherBlockedRepoName) + .get(); final List snapshotStatuses = snapshotsStatusResponse.getSnapshots(); assertThat(snapshotStatuses, hasSize(count)); } @@ -1273,20 +1381,31 @@ private ActionFuture startDeleteFromNonMasterClient(String private ActionFuture startFullSnapshotFromNonMasterClient(String repoName, String snapshotName) { logger.info("--> creating full snapshot [{}] to repo [{}] from non master client", snapshotName, repoName); - return internalCluster().nonMasterClient().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) - .setWaitForCompletion(true).execute(); + return internalCluster().nonMasterClient() + .admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true) + .execute(); } private ActionFuture startFullSnapshotFromMasterClient(String repoName, String snapshotName) { logger.info("--> creating full snapshot [{}] to repo [{}] from master client", snapshotName, repoName); - return internalCluster().masterClient().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) - .setWaitForCompletion(true).execute(); + return internalCluster().masterClient() + .admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true) + .execute(); } private void createIndexWithContent(String indexName, String nodeInclude, String nodeExclude) { - createIndexWithContent(indexName, indexSettingsNoReplicas(1) - .put("index.routing.allocation.include._name", nodeInclude) - .put("index.routing.allocation.exclude._name", nodeExclude).build()); + createIndexWithContent( + indexName, + indexSettingsNoReplicas(1).put("index.routing.allocation.include._name", nodeInclude) + .put("index.routing.allocation.exclude._name", nodeExclude) + .build() + ); } private static boolean snapshotHasCompletedShard(String snapshot, SnapshotsInProgress snapshotsInProgress) { @@ -1310,12 +1429,15 @@ private void corruptIndexN(Path repoPath, long generation) throws IOException { } private static List currentSnapshots(String repoName) { - return client().admin().cluster().prepareGetSnapshots(repoName).setSnapshots(GetSnapshotsRequest.CURRENT_SNAPSHOT) - .get().getSnapshots(repoName); + return client().admin() + .cluster() + .prepareGetSnapshots(repoName) + .setSnapshots(GetSnapshotsRequest.CURRENT_SNAPSHOT) + .get() + .getSnapshots(repoName); } - private ActionFuture startAndBlockOnDeleteSnapshot(String repoName, String snapshotName) - throws Exception { + private ActionFuture startAndBlockOnDeleteSnapshot(String repoName, String snapshotName) throws Exception { final String masterName = internalCluster().getMasterName(); blockNodeOnAnyFiles(repoName, masterName); final ActionFuture fut = startDeleteSnapshot(repoName, snapshotName); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java index 9206b06736d0c..c3fa3a18906a6 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java @@ -62,26 +62,39 @@ public void testConcurrentlyChangeRepositoryContents() throws Exception { Path repo = randomRepoPath(); final String repoName = "test-repo"; - createRepository(repoName, "fs", Settings.builder() - .put("location", repo).put("compress", false) - // Don't cache repository data because the test manually modifies the repository data - .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + repoName, + "fs", + Settings.builder() + .put("location", repo) + .put("compress", false) + // Don't cache repository data because the test manually modifies the repository data + .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx-1", "test-idx-2"); logger.info("--> indexing some data"); - indexRandom(true, + indexRandom( + true, client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); final String snapshot = "test-snap"; logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repoName, snapshot) - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshot) + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); logger.info("--> move index-N blob to next generation"); final RepositoryData repositoryData = getRepositoryData(repoName); @@ -101,40 +114,62 @@ public void testConcurrentlyChangeRepositoryContents() throws Exception { assertAcked(client.admin().cluster().prepareDeleteRepository(repoName)); logger.info("--> recreate repository"); - assertAcked(client.admin().cluster().preparePutRepository(repoName) - .setType("fs").setSettings(Settings.builder() - .put("location", repo) - .put("compress", false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); + assertAcked( + client.admin() + .cluster() + .preparePutRepository(repoName) + .setType("fs") + .setSettings( + Settings.builder() + .put("location", repo) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ) + ); startDeleteSnapshot(repoName, snapshot).get(); logger.info("--> make sure snapshot doesn't exist"); - expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareGetSnapshots(repoName) - .addSnapshots(snapshot).get().getSnapshots(repoName)); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareGetSnapshots(repoName).addSnapshots(snapshot).get().getSnapshots(repoName) + ); } public void testFindDanglingLatestGeneration() throws Exception { Path repo = randomRepoPath(); final String repoName = "test-repo"; - createRepository(repoName, "fs", Settings.builder() - .put("location", repo).put("compress", false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + repoName, + "fs", + Settings.builder() + .put("location", repo) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx-1", "test-idx-2"); logger.info("--> indexing some data"); - indexRandom(true, + indexRandom( + true, client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); final String snapshot = "test-snap"; logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(repoName, snapshot) - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshot) + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); final Repository repository = getRepositoryOnMaster(repoName); @@ -144,10 +179,20 @@ public void testFindDanglingLatestGeneration() throws Exception { Files.move(repo.resolve("index-" + beforeMoveGen), repo.resolve("index-" + (beforeMoveGen + 1))); logger.info("--> set next generation as pending in the cluster state"); - updateClusterState(currentState -> ClusterState.builder(currentState).metadata(Metadata.builder(currentState.getMetadata()) - .putCustom(RepositoriesMetadata.TYPE, - currentState.metadata().custom(RepositoriesMetadata.TYPE).withUpdatedGeneration( - repository.getMetadata().name(), beforeMoveGen, beforeMoveGen + 1)).build()).build()); + updateClusterState( + currentState -> ClusterState.builder(currentState) + .metadata( + Metadata.builder(currentState.getMetadata()) + .putCustom( + RepositoriesMetadata.TYPE, + currentState.metadata() + .custom(RepositoriesMetadata.TYPE) + .withUpdatedGeneration(repository.getMetadata().name(), beforeMoveGen, beforeMoveGen + 1) + ) + .build() + ) + .build() + ); logger.info("--> full cluster restart"); internalCluster().fullRestart(); @@ -162,19 +207,25 @@ public void testFindDanglingLatestGeneration() throws Exception { assertThat(getRepositoryData(repoName).getGenId(), is(beforeMoveGen + 2)); logger.info("--> make sure snapshot doesn't exist"); - expectThrows(SnapshotMissingException.class, () -> client().admin().cluster().prepareGetSnapshots(repoName) - .addSnapshots(snapshot).get().getSnapshots(repoName)); + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repoName).addSnapshots(snapshot).get().getSnapshots(repoName) + ); } public void testHandlingMissingRootLevelSnapshotMetadata() throws Exception { Path repo = randomRepoPath(); final String repoName = "test-repo"; - createRepository(repoName, "fs", Settings.builder() - .put("location", repo) - .put("compress", false) - // Don't cache repository data because the test manually modifies the repository data - .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + repoName, + "fs", + Settings.builder() + .put("location", repo) + .put("compress", false) + // Don't cache repository data because the test manually modifies the repository data + .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); final String snapshotPrefix = "test-snap-"; final int snapshots = randomIntBetween(1, 2); @@ -182,11 +233,17 @@ public void testHandlingMissingRootLevelSnapshotMetadata() throws Exception { for (int i = 0; i < snapshots; ++i) { // Workaround to simulate BwC situation: taking a snapshot without indices here so that we don't create any new version shard // generations (the existence of which would short-circuit checks for the repo containing old version snapshots) - CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(repoName, snapshotPrefix + i) - .setIndices().setWaitForCompletion(true).get(); + CreateSnapshotResponse createSnapshotResponse = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotPrefix + i) + .setIndices() + .setWaitForCompletion(true) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), is(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); } final RepositoryData repositoryData = getRepositoryData(repoName); @@ -196,41 +253,63 @@ public void testHandlingMissingRootLevelSnapshotMetadata() throws Exception { logger.info("--> strip version information from index-N blob"); final RepositoryData withoutVersions = new RepositoryData( - RepositoryData.MISSING_UUID, // old-format repository data has no UUID - repositoryData.getGenId(), - repositoryData.getSnapshotIds().stream().collect(Collectors.toMap(SnapshotId::getUUID, Function.identity())), - repositoryData.getSnapshotIds().stream().collect(Collectors.toMap(SnapshotId::getUUID, - s -> new RepositoryData.SnapshotDetails( - repositoryData.getSnapshotState(s), - null, - -1, - -1))), - Collections.emptyMap(), - ShardGenerations.EMPTY, - IndexMetaDataGenerations.EMPTY, - RepositoryData.MISSING_UUID); // old-format repository has no cluster UUID - - Files.write(repo.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + withoutVersions.getGenId()), - BytesReference.toBytes(BytesReference.bytes( - withoutVersions.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT, true))), - StandardOpenOption.TRUNCATE_EXISTING); + RepositoryData.MISSING_UUID, // old-format repository data has no UUID + repositoryData.getGenId(), + repositoryData.getSnapshotIds().stream().collect(Collectors.toMap(SnapshotId::getUUID, Function.identity())), + repositoryData.getSnapshotIds() + .stream() + .collect( + Collectors.toMap( + SnapshotId::getUUID, + s -> new RepositoryData.SnapshotDetails(repositoryData.getSnapshotState(s), null, -1, -1) + ) + ), + Collections.emptyMap(), + ShardGenerations.EMPTY, + IndexMetaDataGenerations.EMPTY, + RepositoryData.MISSING_UUID + ); // old-format repository has no cluster UUID + + Files.write( + repo.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + withoutVersions.getGenId()), + BytesReference.toBytes( + BytesReference.bytes(withoutVersions.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT, true)) + ), + StandardOpenOption.TRUNCATE_EXISTING + ); logger.info("--> verify that repo is assumed in old metadata format"); final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); final ThreadPool threadPool = internalCluster().getCurrentMasterNodeInstance(ThreadPool.class); - assertThat(PlainActionFuture.get(f -> threadPool.generic().execute( - ActionRunnable.supply(f, () -> - snapshotsService.minCompatibleVersion(Version.CURRENT, getRepositoryData(repoName), null)))), - is(SnapshotsService.OLD_SNAPSHOT_FORMAT)); + assertThat( + PlainActionFuture.get( + f -> threadPool.generic() + .execute( + ActionRunnable.supply( + f, + () -> snapshotsService.minCompatibleVersion(Version.CURRENT, getRepositoryData(repoName), null) + ) + ) + ), + is(SnapshotsService.OLD_SNAPSHOT_FORMAT) + ); logger.info("--> verify that snapshot with missing root level metadata can be deleted"); assertAcked(startDeleteSnapshot(repoName, snapshotToCorrupt.getName()).get()); logger.info("--> verify that repository is assumed in new metadata format after removing corrupted snapshot"); - assertThat(PlainActionFuture.get(f -> threadPool.generic().execute( - ActionRunnable.supply(f, () -> - snapshotsService.minCompatibleVersion(Version.CURRENT, getRepositoryData(repoName), null)))), - is(Version.CURRENT)); + assertThat( + PlainActionFuture.get( + f -> threadPool.generic() + .execute( + ActionRunnable.supply( + f, + () -> snapshotsService.minCompatibleVersion(Version.CURRENT, getRepositoryData(repoName), null) + ) + ) + ), + is(Version.CURRENT) + ); final RepositoryData finalRepositoryData = getRepositoryData(repoName); for (SnapshotId snapshotId : finalRepositoryData.getSnapshotIds()) { assertThat(finalRepositoryData.getVersion(snapshotId), is(Version.CURRENT)); @@ -243,19 +322,29 @@ public void testMountCorruptedRepositoryData() throws Exception { Path repo = randomRepoPath(); final String repoName = "test-repo"; - createRepository(repoName, "fs", Settings.builder() - .put("location", repo) - // Don't cache repository data because the test manually modifies the repository data - .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) - .put("compress", false)); + createRepository( + repoName, + "fs", + Settings.builder() + .put("location", repo) + // Don't cache repository data because the test manually modifies the repository data + .put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) + .put("compress", false) + ); final String snapshot = "test-snap"; logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repoName, snapshot) - .setWaitForCompletion(true).setIndices("test-idx-*").get(); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshot) + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); logger.info("--> corrupt index-N blob"); final Repository repository = getRepositoryOnMaster(repoName); @@ -266,10 +355,12 @@ public void testMountCorruptedRepositoryData() throws Exception { expectThrows(RepositoryException.class, () -> getRepositoryData(repository)); final String otherRepoName = "other-repo"; - assertAcked(clusterAdmin().preparePutRepository(otherRepoName) - .setType("fs") - .setVerify(false) // don't try and load the repo data, since it is corrupt - .setSettings(Settings.builder().put("location", repo).put("compress", false))); + assertAcked( + clusterAdmin().preparePutRepository(otherRepoName) + .setType("fs") + .setVerify(false) // don't try and load the repo data, since it is corrupt + .setSettings(Settings.builder().put("location", repo).put("compress", false)) + ); final Repository otherRepo = getRepositoryOnMaster(otherRepoName); logger.info("--> verify loading repository data from newly mounted repository throws RepositoryException"); @@ -330,20 +421,26 @@ public void testRepairBrokenShardGenerations() throws Exception { Files.move(initialShardMetaPath, shardPath.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + randomIntBetween(1, 1000))); final RepositoryData repositoryData = getRepositoryData(repoName); - final Map snapshotIds = - repositoryData.getSnapshotIds().stream().collect(Collectors.toMap(SnapshotId::getUUID, Function.identity())); + final Map snapshotIds = repositoryData.getSnapshotIds() + .stream() + .collect(Collectors.toMap(SnapshotId::getUUID, Function.identity())); final RepositoryData brokenRepoData = new RepositoryData( - repositoryData.getUuid(), - repositoryData.getGenId(), - snapshotIds, - snapshotIds.values().stream().collect(Collectors.toMap(SnapshotId::getUUID, repositoryData::getSnapshotDetails)), - repositoryData.getIndices().values().stream().collect(Collectors.toMap(Function.identity(), repositoryData::getSnapshots)), - ShardGenerations.builder().putAll(repositoryData.shardGenerations()).put(indexId, 0, "0").build(), - repositoryData.indexMetaDataGenerations(), repositoryData.getClusterUUID()); - Files.write(repoPath.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + repositoryData.getGenId()), - BytesReference.toBytes(BytesReference.bytes( - brokenRepoData.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT))), - StandardOpenOption.TRUNCATE_EXISTING); + repositoryData.getUuid(), + repositoryData.getGenId(), + snapshotIds, + snapshotIds.values().stream().collect(Collectors.toMap(SnapshotId::getUUID, repositoryData::getSnapshotDetails)), + repositoryData.getIndices().values().stream().collect(Collectors.toMap(Function.identity(), repositoryData::getSnapshots)), + ShardGenerations.builder().putAll(repositoryData.shardGenerations()).put(indexId, 0, "0").build(), + repositoryData.indexMetaDataGenerations(), + repositoryData.getClusterUUID() + ); + Files.write( + repoPath.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + repositoryData.getGenId()), + BytesReference.toBytes( + BytesReference.bytes(brokenRepoData.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT)) + ), + StandardOpenOption.TRUNCATE_EXISTING + ); logger.info("--> recreating repository to clear caches"); client().admin().cluster().prepareDeleteRepository(repoName).get(); @@ -383,8 +480,9 @@ public void testSnapshotWithCorruptedShardIndexFile() throws Exception { final IndexId corruptedIndex = indexIds.get(indexName); final Path shardIndexFile = repo.resolve("indices") - .resolve(corruptedIndex.getId()).resolve("0") - .resolve("index-" + repositoryData.shardGenerations().getShardGen(corruptedIndex, 0)); + .resolve(corruptedIndex.getId()) + .resolve("0") + .resolve("index-" + repositoryData.shardGenerations().getShardGen(corruptedIndex, 0)); logger.info("--> truncating shard index file [{}]", shardIndexFile); try (SeekableByteChannel outChan = Files.newByteChannel(shardIndexFile, StandardOpenOption.WRITE)) { @@ -402,9 +500,9 @@ public void testSnapshotWithCorruptedShardIndexFile() throws Exception { logger.info("--> restoring snapshot [{}]", snapshot1); clusterAdmin().prepareRestoreSnapshot("test-repo", snapshot1) - .setRestoreGlobalState(randomBoolean()) - .setWaitForCompletion(true) - .get(); + .setRestoreGlobalState(randomBoolean()) + .setWaitForCompletion(true) + .get(); ensureGreen(); assertDocCount(indexName, nDocs); @@ -418,9 +516,9 @@ public void testSnapshotWithCorruptedShardIndexFile() throws Exception { final String snapshot2 = "test-snap-2"; logger.info("--> creating snapshot [{}]", snapshot2); final SnapshotInfo snapshotInfo2 = clusterAdmin().prepareCreateSnapshot("test-repo", snapshot2) - .setWaitForCompletion(true) - .get() - .getSnapshotInfo(); + .setWaitForCompletion(true) + .get() + .getSnapshotInfo(); assertThat(snapshotInfo2.state(), equalTo(SnapshotState.PARTIAL)); assertThat(snapshotInfo2.failedShards(), equalTo(1)); assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo.totalShards() - 1)); @@ -433,16 +531,22 @@ public void testDeleteSnapshotWithMissingIndexAndShardMetadata() throws Exceptio Path repo = randomRepoPath(); createRepository("test-repo", "fs", repo); - final String[] indices = {"test-idx-1", "test-idx-2"}; + final String[] indices = { "test-idx-1", "test-idx-2" }; createIndex(indices); logger.info("--> indexing some data"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices(indices).get(); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap-1") + .setWaitForCompletion(true) + .setIndices(indices) + .get(); final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards())); @@ -454,8 +558,9 @@ public void testDeleteSnapshotWithMissingIndexAndShardMetadata() throws Exceptio for (String index : indices) { Path shardZero = indicesPath.resolve(indexIds.get(index).getId()).resolve("0"); if (randomBoolean()) { - Files.delete(shardZero.resolve("index-" + getRepositoryData("test-repo").shardGenerations() - .getShardGen(indexIds.get(index), 0))); + Files.delete( + shardZero.resolve("index-" + getRepositoryData("test-repo").shardGenerations().getShardGen(indexIds.get(index), 0)) + ); } Files.delete(shardZero.resolve("snap-" + snapshotInfo.snapshotId().getUUID() + ".dat")); } @@ -464,8 +569,10 @@ public void testDeleteSnapshotWithMissingIndexAndShardMetadata() throws Exceptio logger.info("--> make sure snapshot doesn't exist"); - expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareGetSnapshots("test-repo") - .addSnapshots("test-snap-1").get().getSnapshots("test-repo")); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("test-snap-1").get().getSnapshots("test-repo") + ); for (String index : indices) { assertTrue(Files.notExists(indicesPath.resolve(indexIds.get(index).getId()))); @@ -480,16 +587,24 @@ public void testDeleteSnapshotWithMissingMetadata() throws Exception { createIndex("test-idx-1", "test-idx-2"); logger.info("--> indexing some data"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap-1") + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); logger.info("--> delete global state metadata"); Path metadata = repo.resolve("meta-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat"); @@ -498,64 +613,90 @@ public void testDeleteSnapshotWithMissingMetadata() throws Exception { startDeleteSnapshot("test-repo", "test-snap-1").get(); logger.info("--> make sure snapshot doesn't exist"); - expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareGetSnapshots("test-repo") - .addSnapshots("test-snap-1").get().getSnapshots("test-repo")); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("test-snap-1").get().getSnapshots("test-repo") + ); } public void testDeleteSnapshotWithCorruptedSnapshotFile() throws Exception { Client client = client(); Path repo = randomRepoPath(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", repo).put("compress", false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + "test-repo", + "fs", + Settings.builder() + .put("location", repo) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx-1", "test-idx-2"); logger.info("--> indexing some data"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); logger.info("--> creating snapshot"); - CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap-1") + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); logger.info("--> truncate snapshot file to make it unreadable"); Path snapshotPath = repo.resolve("snap-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat"); - try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) { + try (SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) { outChan.truncate(randomInt(10)); } startDeleteSnapshot("test-repo", "test-snap-1").get(); logger.info("--> make sure snapshot doesn't exist"); - expectThrows(SnapshotMissingException.class, - () -> client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("test-snap-1").get(). - getSnapshots("test-repo")); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("test-snap-1").get().getSnapshots("test-repo") + ); logger.info("--> make sure that we can create the snapshot again"); - createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap-1") + .setWaitForCompletion(true) + .setIndices("test-idx-*") + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); } /** Tests that a snapshot with a corrupted global state file can still be deleted */ public void testDeleteSnapshotWithCorruptedGlobalState() throws Exception { final Path repo = randomRepoPath(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", repo) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + "test-repo", + "fs", + Settings.builder().put("location", repo).put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx-1", "test-idx-2"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); flushAndRefresh("test-idx-1", "test-idx-2"); SnapshotInfo snapshotInfo = createFullSnapshot("test-repo", "test-snap"); @@ -576,16 +717,19 @@ public void testDeleteSnapshotWithCorruptedGlobalState() throws Exception { assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo("test-snap")); - SnapshotsStatusResponse snapshotStatusResponse = - clusterAdmin().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get(); + SnapshotsStatusResponse snapshotStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get(); assertThat(snapshotStatusResponse.getSnapshots(), hasSize(1)); assertThat(snapshotStatusResponse.getSnapshots().get(0).getSnapshot().getSnapshotId().getName(), equalTo("test-snap")); assertAcked(startDeleteSnapshot("test-repo", "test-snap").get()); - expectThrows(SnapshotMissingException.class, () -> clusterAdmin() - .prepareGetSnapshots("test-repo").addSnapshots("test-snap").get().getSnapshots("test-repo")); - assertRequestBuilderThrows(clusterAdmin().prepareSnapshotStatus("test-repo").addSnapshots("test-snap"), - SnapshotMissingException.class); + expectThrows( + SnapshotMissingException.class, + () -> clusterAdmin().prepareGetSnapshots("test-repo").addSnapshots("test-snap").get().getSnapshots("test-repo") + ); + assertRequestBuilderThrows( + clusterAdmin().prepareSnapshotStatus("test-repo").addSnapshots("test-snap"), + SnapshotMissingException.class + ); createFullSnapshot("test-repo", "test-snap"); } @@ -598,13 +742,14 @@ public void testSnapshotWithMissingShardLevelIndexFile() throws Exception { createIndex("test-idx-1", "test-idx-2"); logger.info("--> indexing some data"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); logger.info("--> creating snapshot"); - clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices("test-idx-*").get(); + clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-1").setWaitForCompletion(true).setIndices("test-idx-*").get(); logger.info("--> deleting shard level index file"); final Path indicesPath = repo.resolve("indices"); @@ -612,37 +757,52 @@ public void testSnapshotWithMissingShardLevelIndexFile() throws Exception { final Path shardGen; try (Stream shardFiles = Files.list(indicesPath.resolve(indexId.getId()).resolve("0"))) { shardGen = shardFiles.filter(file -> file.getFileName().toString().startsWith(BlobStoreRepository.INDEX_FILE_PREFIX)) - .findFirst().orElseThrow(() -> new AssertionError("Failed to find shard index blob")); + .findFirst() + .orElseThrow(() -> new AssertionError("Failed to find shard index blob")); } Files.delete(shardGen); } logger.info("--> creating another snapshot"); - CreateSnapshotResponse createSnapshotResponse = - clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2") - .setWaitForCompletion(true).setIndices("test-idx-1").get(); - assertEquals(createSnapshotResponse.getSnapshotInfo().successfulShards(), - createSnapshotResponse.getSnapshotInfo().totalShards() - 1); - - logger.info("--> restoring the first snapshot, the repository should not have lost any shard data despite deleting index-N, " + - "because it uses snap-*.data files and not the index-N to determine what files to restore"); + CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2") + .setWaitForCompletion(true) + .setIndices("test-idx-1") + .get(); + assertEquals( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + createSnapshotResponse.getSnapshotInfo().totalShards() - 1 + ); + + logger.info( + "--> restoring the first snapshot, the repository should not have lost any shard data despite deleting index-N, " + + "because it uses snap-*.data files and not the index-N to determine what files to restore" + ); client().admin().indices().prepareDelete("test-idx-1", "test-idx-2").get(); - RestoreSnapshotResponse restoreSnapshotResponse = - clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-1").setWaitForCompletion(true).get(); + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-1") + .setWaitForCompletion(true) + .get(); assertEquals(0, restoreSnapshotResponse.getRestoreInfo().failedShards()); } private void assertRepositoryBlocked(Client client, String repo, String existingSnapshot) { logger.info("--> try to delete snapshot"); - final RepositoryException repositoryException3 = expectThrows(RepositoryException.class, - () -> client.admin().cluster().prepareDeleteSnapshot(repo, existingSnapshot).execute().actionGet()); - assertThat(repositoryException3.getMessage(), - containsString("Could not read repository data because the contents of the repository do not match its expected state.")); + final RepositoryException repositoryException3 = expectThrows( + RepositoryException.class, + () -> client.admin().cluster().prepareDeleteSnapshot(repo, existingSnapshot).execute().actionGet() + ); + assertThat( + repositoryException3.getMessage(), + containsString("Could not read repository data because the contents of the repository do not match its expected state.") + ); logger.info("--> try to create snapshot"); - final RepositoryException repositoryException4 = expectThrows(RepositoryException.class, - () -> client.admin().cluster().prepareCreateSnapshot(repo, existingSnapshot).execute().actionGet()); - assertThat(repositoryException4.getMessage(), - containsString("Could not read repository data because the contents of the repository do not match its expected state.")); + final RepositoryException repositoryException4 = expectThrows( + RepositoryException.class, + () -> client.admin().cluster().prepareCreateSnapshot(repo, existingSnapshot).execute().actionGet() + ); + assertThat( + repositoryException4.getMessage(), + containsString("Could not read repository data because the contents of the repository do not match its expected state.") + ); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CustomMetadataSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CustomMetadataSnapshotIT.java index 1447aac720e37..60d301739aa6b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CustomMetadataSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CustomMetadataSnapshotIT.java @@ -57,8 +57,10 @@ public void testRestoreCustomMetadata() throws Exception { metadataBuilder.putCustom(NonSnapshottableMetadata.TYPE, new NonSnapshottableMetadata("before_snapshot_ns")); metadataBuilder.putCustom(SnapshottableGatewayMetadata.TYPE, new SnapshottableGatewayMetadata("before_snapshot_s_gw")); metadataBuilder.putCustom(NonSnapshottableGatewayMetadata.TYPE, new NonSnapshottableGatewayMetadata("before_snapshot_ns_gw")); - metadataBuilder.putCustom(SnapshotableGatewayNoApiMetadata.TYPE, - new SnapshotableGatewayNoApiMetadata("before_snapshot_s_gw_noapi")); + metadataBuilder.putCustom( + SnapshotableGatewayNoApiMetadata.TYPE, + new SnapshotableGatewayNoApiMetadata("before_snapshot_s_gw_noapi") + ); builder.metadata(metadataBuilder); return builder.build(); }); @@ -94,8 +96,12 @@ public void testRestoreCustomMetadata() throws Exception { createRepository("test-repo-2", "fs", tempDir); logger.info("--> restore snapshot"); - clusterAdmin().prepareRestoreSnapshot("test-repo-2", "test-snap").setRestoreGlobalState(true).setIndices("-*") - .setWaitForCompletion(true).execute().actionGet(); + clusterAdmin().prepareRestoreSnapshot("test-repo-2", "test-snap") + .setRestoreGlobalState(true) + .setIndices("-*") + .setWaitForCompletion(true) + .execute() + .actionGet(); logger.info("--> make sure old repository wasn't restored"); assertRequestBuilderThrows(clusterAdmin().prepareGetRepositories("test-repo"), RepositoryMissingException.class); @@ -107,10 +113,14 @@ public void testRestoreCustomMetadata() throws Exception { Metadata metadata = clusterState.getMetadata(); assertThat(((SnapshottableMetadata) metadata.custom(SnapshottableMetadata.TYPE)).getData(), equalTo("before_snapshot_s")); assertThat(((NonSnapshottableMetadata) metadata.custom(NonSnapshottableMetadata.TYPE)).getData(), equalTo("after_snapshot_ns")); - assertThat(((SnapshottableGatewayMetadata) metadata.custom(SnapshottableGatewayMetadata.TYPE)).getData(), - equalTo("before_snapshot_s_gw")); - assertThat(((NonSnapshottableGatewayMetadata) metadata.custom(NonSnapshottableGatewayMetadata.TYPE)).getData(), - equalTo("after_snapshot_ns_gw")); + assertThat( + ((SnapshottableGatewayMetadata) metadata.custom(SnapshottableGatewayMetadata.TYPE)).getData(), + equalTo("before_snapshot_s_gw") + ); + assertThat( + ((NonSnapshottableGatewayMetadata) metadata.custom(NonSnapshottableGatewayMetadata.TYPE)).getData(), + equalTo("after_snapshot_ns_gw") + ); logger.info("--> restart all nodes"); internalCluster().fullRestart(); @@ -122,16 +132,22 @@ public void testRestoreCustomMetadata() throws Exception { metadata = clusterState.getMetadata(); assertThat(metadata.custom(SnapshottableMetadata.TYPE), nullValue()); assertThat(metadata.custom(NonSnapshottableMetadata.TYPE), nullValue()); - assertThat(((SnapshottableGatewayMetadata) metadata.custom(SnapshottableGatewayMetadata.TYPE)).getData(), - equalTo("before_snapshot_s_gw")); - assertThat(((NonSnapshottableGatewayMetadata) metadata.custom(NonSnapshottableGatewayMetadata.TYPE)).getData(), - equalTo("after_snapshot_ns_gw")); + assertThat( + ((SnapshottableGatewayMetadata) metadata.custom(SnapshottableGatewayMetadata.TYPE)).getData(), + equalTo("before_snapshot_s_gw") + ); + assertThat( + ((NonSnapshottableGatewayMetadata) metadata.custom(NonSnapshottableGatewayMetadata.TYPE)).getData(), + equalTo("after_snapshot_ns_gw") + ); // Shouldn't be returned as part of API response assertThat(metadata.custom(SnapshotableGatewayNoApiMetadata.TYPE), nullValue()); // But should still be in state metadata = internalCluster().getInstance(ClusterService.class).state().metadata(); - assertThat(((SnapshotableGatewayNoApiMetadata) metadata.custom(SnapshotableGatewayNoApiMetadata.TYPE)).getData(), - equalTo("before_snapshot_s_gw_noapi")); + assertThat( + ((SnapshotableGatewayNoApiMetadata) metadata.custom(SnapshotableGatewayNoApiMetadata.TYPE)).getData(), + equalTo("before_snapshot_s_gw_noapi") + ); } public static class TestCustomMetadataPlugin extends Plugin { @@ -143,25 +159,48 @@ public TestCustomMetadataPlugin() { registerBuiltinWritables(); } - private void registerMetadataCustom(String name, Writeable.Reader reader, - Writeable.Reader diffReader, - CheckedFunction parser) { + private void registerMetadataCustom( + String name, + Writeable.Reader reader, + Writeable.Reader diffReader, + CheckedFunction parser + ) { namedWritables.add(new NamedWriteableRegistry.Entry(Metadata.Custom.class, name, reader)); namedWritables.add(new NamedWriteableRegistry.Entry(NamedDiff.class, name, diffReader)); namedXContents.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(name), parser)); } private void registerBuiltinWritables() { - registerMetadataCustom(SnapshottableMetadata.TYPE, SnapshottableMetadata::readFrom, - SnapshottableMetadata::readDiffFrom, SnapshottableMetadata::fromXContent); - registerMetadataCustom(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata::readFrom, - NonSnapshottableMetadata::readDiffFrom, NonSnapshottableMetadata::fromXContent); - registerMetadataCustom(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata::readFrom, - SnapshottableGatewayMetadata::readDiffFrom, SnapshottableGatewayMetadata::fromXContent); - registerMetadataCustom(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata::readFrom, - NonSnapshottableGatewayMetadata::readDiffFrom, NonSnapshottableGatewayMetadata::fromXContent); - registerMetadataCustom(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata::readFrom, - NonSnapshottableGatewayMetadata::readDiffFrom, SnapshotableGatewayNoApiMetadata::fromXContent); + registerMetadataCustom( + SnapshottableMetadata.TYPE, + SnapshottableMetadata::readFrom, + SnapshottableMetadata::readDiffFrom, + SnapshottableMetadata::fromXContent + ); + registerMetadataCustom( + NonSnapshottableMetadata.TYPE, + NonSnapshottableMetadata::readFrom, + NonSnapshottableMetadata::readDiffFrom, + NonSnapshottableMetadata::fromXContent + ); + registerMetadataCustom( + SnapshottableGatewayMetadata.TYPE, + SnapshottableGatewayMetadata::readFrom, + SnapshottableGatewayMetadata::readDiffFrom, + SnapshottableGatewayMetadata::fromXContent + ); + registerMetadataCustom( + NonSnapshottableGatewayMetadata.TYPE, + NonSnapshottableGatewayMetadata::readFrom, + NonSnapshottableGatewayMetadata::readDiffFrom, + NonSnapshottableGatewayMetadata::fromXContent + ); + registerMetadataCustom( + SnapshotableGatewayNoApiMetadata.TYPE, + SnapshotableGatewayNoApiMetadata::readFrom, + NonSnapshottableGatewayMetadata::readDiffFrom, + SnapshotableGatewayNoApiMetadata::fromXContent + ); } @Override diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index cc38ee58613e8..c3db7e93be837 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -112,19 +112,18 @@ public void testSnapshotDuringNodeShutdown() throws Exception { indexRandomDocs("test-idx", 100); final Path repoPath = randomRepoPath(); - createRepository("test-repo", "mock", - Settings.builder().put("location", repoPath).put("random", randomAlphaOfLength(10)) - .put("wait_after_unblock", 200)); + createRepository( + "test-repo", + "mock", + Settings.builder().put("location", repoPath).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); maybeInitWithOldSnapshotVersion("test-repo", repoPath); // Pick one node and block it String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); logger.info("--> snapshot"); - clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(false) - .setIndices("test-idx") - .get(); + clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); logger.info("--> waiting for block to kick in"); waitForBlock(blockedNode, "test-repo"); @@ -149,9 +148,11 @@ public void testSnapshotWithStuckNode() throws Exception { indexRandomDocs("test-idx", 100); Path repo = randomRepoPath(); - createRepository("test-repo", "mock", - Settings.builder().put("location", repo).put("random", randomAlphaOfLength(10)) - .put("wait_after_unblock", 200)); + createRepository( + "test-repo", + "mock", + Settings.builder().put("location", repo).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); // Pick one node and block it String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); @@ -160,10 +161,7 @@ public void testSnapshotWithStuckNode() throws Exception { assertFileCount(repo, 0); logger.info("--> snapshot"); - clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(false) - .setIndices("test-idx") - .get(); + clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); logger.info("--> waiting for block to kick in"); waitForBlock(blockedNode, "test-repo"); @@ -171,7 +169,10 @@ public void testSnapshotWithStuckNode() throws Exception { logger.info("--> execution was blocked on node [{}], aborting snapshot", blockedNode); ActionFuture deleteSnapshotResponseFuture = internalCluster().client(nodes.get(0)) - .admin().cluster().prepareDeleteSnapshot("test-repo", "test-snap").execute(); + .admin() + .cluster() + .prepareDeleteSnapshot("test-repo", "test-snap") + .execute(); // Make sure that abort makes some progress Thread.sleep(100); unblockNode("test-repo", blockedNode); @@ -186,16 +187,17 @@ public void testSnapshotWithStuckNode() throws Exception { } logger.info("--> making sure that snapshot no longer exists"); - expectThrows(SnapshotMissingException.class, - () -> clusterAdmin().prepareGetSnapshots("test-repo").setSnapshots("test-snap") - .execute().actionGet().getSnapshots("test-repo")); + expectThrows( + SnapshotMissingException.class, + () -> clusterAdmin().prepareGetSnapshots("test-repo").setSnapshots("test-snap").execute().actionGet().getSnapshots("test-repo") + ); logger.info("--> Go through a loop of creating and deleting a snapshot to trigger repository cleanup"); clusterAdmin().prepareCleanupRepository("test-repo").get(); // Expect two files to remain in the repository: - // (1) index-(N+1) - // (2) index-latest + // (1) index-(N+1) + // (2) index-latest assertFileCount(repo, 2); logger.info("--> done"); } @@ -213,9 +215,16 @@ public void testRestoreIndexWithMissingShards() throws Exception { logger.info("--> shutdown one of the nodes"); internalCluster().stopRandomDataNode(); - assertThat(clusterAdmin().prepareHealth().setWaitForEvents(Priority.LANGUID).setTimeout("1m").setWaitForNodes("<2") - .execute().actionGet().isTimedOut(), - equalTo(false)); + assertThat( + clusterAdmin().prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout("1m") + .setWaitForNodes("<2") + .execute() + .actionGet() + .isTimedOut(), + equalTo(false) + ); logger.info("--> create an index that will have all allocated shards"); assertAcked(prepareCreate("test-idx-all", 1, indexSettingsNoReplicas(6))); @@ -228,34 +237,46 @@ public void testRestoreIndexWithMissingShards() throws Exception { assertAcked(client().admin().indices().prepareClose("test-idx-closed")); logger.info("--> create an index that will have no allocated shards"); - assertAcked(prepareCreate("test-idx-none", 1, indexSettingsNoReplicas(6) - .put("index.routing.allocation.include.tag", "nowhere")).setWaitForActiveShards(ActiveShardCount.NONE).get()); + assertAcked( + prepareCreate("test-idx-none", 1, indexSettingsNoReplicas(6).put("index.routing.allocation.include.tag", "nowhere")) + .setWaitForActiveShards(ActiveShardCount.NONE) + .get() + ); assertTrue(indexExists("test-idx-none")); createRepository("test-repo", "fs"); logger.info("--> start snapshot with default settings without a closed index - should fail"); - final SnapshotException sne = expectThrows(SnapshotException.class, + final SnapshotException sne = expectThrows( + SnapshotException.class, () -> clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-1") .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") - .setWaitForCompletion(true).execute().actionGet()); + .setWaitForCompletion(true) + .execute() + .actionGet() + ); assertThat(sne.getMessage(), containsString("Indices don't have primary shards")); if (randomBoolean()) { logger.info("checking snapshot completion using status"); clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2") - .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") - .setWaitForCompletion(false).setPartial(true).execute().actionGet(); + .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") + .setWaitForCompletion(false) + .setPartial(true) + .execute() + .actionGet(); assertBusy(() -> { SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo") - .setSnapshots("test-snap-2").get(); + .setSnapshots("test-snap-2") + .get(); List snapshotStatuses = snapshotsStatusResponse.getSnapshots(); assertEquals(snapshotStatuses.size(), 1); logger.trace("current snapshot status [{}]", snapshotStatuses.get(0)); assertTrue(snapshotStatuses.get(0).getState().completed()); }, 1, TimeUnit.MINUTES); SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo") - .setSnapshots("test-snap-2").get(); + .setSnapshots("test-snap-2") + .get(); List snapshotStatuses = snapshotsStatusResponse.getSnapshots(); assertThat(snapshotStatuses.size(), equalTo(1)); SnapshotStatus snapshotStatus = snapshotStatuses.get(0); @@ -273,29 +294,41 @@ public void testRestoreIndexWithMissingShards() throws Exception { }, 1, TimeUnit.MINUTES); } else { logger.info("checking snapshot completion using wait_for_completion flag"); - final CreateSnapshotResponse createSnapshotResponse = - clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2") - .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") - .setWaitForCompletion(true).setPartial(true).execute().actionGet(); - logger.info("State: [{}], Reason: [{}]", - createSnapshotResponse.getSnapshotInfo().state(), createSnapshotResponse.getSnapshotInfo().reason()); + final CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2") + .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") + .setWaitForCompletion(true) + .setPartial(true) + .execute() + .actionGet(); + logger.info( + "State: [{}], Reason: [{}]", + createSnapshotResponse.getSnapshotInfo().state(), + createSnapshotResponse.getSnapshotInfo().reason() + ); assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), equalTo(22)); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), lessThan(16)); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(10)); - assertThat(getSnapshot("test-repo", "test-snap-2").state(), - equalTo(SnapshotState.PARTIAL)); + assertThat(getSnapshot("test-repo", "test-snap-2").state(), equalTo(SnapshotState.PARTIAL)); } assertAcked(client().admin().indices().prepareClose("test-idx-all")); logger.info("--> restore incomplete snapshot - should fail"); - assertFutureThrows(clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2").setRestoreGlobalState(false) - .setWaitForCompletion(true).execute(), - SnapshotRestoreException.class); + assertFutureThrows( + clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2") + .setRestoreGlobalState(false) + .setWaitForCompletion(true) + .execute(), + SnapshotRestoreException.class + ); logger.info("--> restore snapshot for the index that was snapshotted completely"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2") - .setRestoreGlobalState(false).setIndices("test-idx-all").setWaitForCompletion(true).execute().actionGet(); + .setRestoreGlobalState(false) + .setIndices("test-idx-all") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo(), notNullValue()); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(6)); assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(6)); @@ -305,7 +338,11 @@ public void testRestoreIndexWithMissingShards() throws Exception { logger.info("--> restore snapshot for the partial index"); cluster().wipeIndices("test-idx-some"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2") - .setRestoreGlobalState(false).setIndices("test-idx-some").setPartial(true).setWaitForCompletion(true).get(); + .setRestoreGlobalState(false) + .setIndices("test-idx-some") + .setPartial(true) + .setWaitForCompletion(true) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo(), notNullValue()); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(6)); assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0), lessThan(6))); @@ -315,7 +352,11 @@ public void testRestoreIndexWithMissingShards() throws Exception { logger.info("--> restore snapshot for the index that didn't have any shards snapshotted successfully"); cluster().wipeIndices("test-idx-none"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2") - .setRestoreGlobalState(false).setIndices("test-idx-none").setPartial(true).setWaitForCompletion(true).get(); + .setRestoreGlobalState(false) + .setIndices("test-idx-none") + .setPartial(true) + .setWaitForCompletion(true) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo(), notNullValue()); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(6)); assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(0)); @@ -324,7 +365,11 @@ public void testRestoreIndexWithMissingShards() throws Exception { logger.info("--> restore snapshot for the closed index that was snapshotted completely"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-2") - .setRestoreGlobalState(false).setIndices("test-idx-closed").setWaitForCompletion(true).execute().actionGet(); + .setRestoreGlobalState(false) + .setIndices("test-idx-closed") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo(), notNullValue()); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(4)); assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(4)); @@ -346,8 +391,10 @@ public void testRestoreIndexWithShardsMissingInLocalGateway() throws Exception { indexRandomDocs("test-idx", 100); logger.info("--> force merging down to a single segment to get a deterministic set of files"); - assertEquals(client().admin().indices().prepareForceMerge("test-idx").setMaxNumSegments(1).setFlush(true).get().getFailedShards(), - 0); + assertEquals( + client().admin().indices().prepareForceMerge("test-idx").setMaxNumSegments(1).setFlush(true).get().getFailedShards(), + 0 + ); createSnapshot("test-repo", "test-snap-1", Collections.singletonList("test-idx")); @@ -362,20 +409,37 @@ public boolean clearData(String nodeName) { } }); - assertThat(clusterAdmin().prepareHealth().setWaitForEvents(Priority.LANGUID).setTimeout("1m").setWaitForNodes("2") - .execute().actionGet().isTimedOut(), - equalTo(false)); + assertThat( + clusterAdmin().prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout("1m") + .setWaitForNodes("2") + .execute() + .actionGet() + .isTimedOut(), + equalTo(false) + ); logger.info("--> restore index snapshot"); - assertThat(clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-1").setRestoreGlobalState(false) - .setWaitForCompletion(true).get().getRestoreInfo().successfulShards(), - equalTo(6)); + assertThat( + clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-1") + .setRestoreGlobalState(false) + .setWaitForCompletion(true) + .get() + .getRestoreInfo() + .successfulShards(), + equalTo(6) + ); ensureGreen("test-idx"); IntSet reusedShards = new IntHashSet(); - List recoveryStates = client().admin().indices().prepareRecoveries("test-idx").get() - .shardRecoveryStates().get("test-idx"); + List recoveryStates = client().admin() + .indices() + .prepareRecoveries("test-idx") + .get() + .shardRecoveryStates() + .get("test-idx"); for (RecoveryState recoveryState : recoveryStates) { if (recoveryState.getIndex().reusedBytes() > 0) { reusedShards.add(recoveryState.getShardId().getId()); @@ -395,13 +459,16 @@ public void testRegistrationFailure() { // Register mock repositories for (int i = 0; i < 5; i++) { clusterAdmin().preparePutRepository("test-repo" + i) - .setType("mock").setSettings(Settings.builder() - .put("location", randomRepoPath())).setVerify(false).get(); + .setType("mock") + .setSettings(Settings.builder().put("location", randomRepoPath())) + .setVerify(false) + .get(); } logger.info("--> make sure that properly setup repository can be registered on all nodes"); clusterAdmin().preparePutRepository("test-repo-0") - .setType("fs").setSettings(Settings.builder() - .put("location", randomRepoPath())).get(); + .setType("fs") + .setSettings(Settings.builder().put("location", randomRepoPath())) + .get(); } @@ -409,10 +476,14 @@ public void testThatSensitiveRepositorySettingsAreNotExposed() throws Exception disableRepoConsistencyCheck("This test does not create any data in the repository"); logger.info("--> start two nodes"); internalCluster().startNodes(2); - createRepository("test-repo", "mock", Settings.builder() - .put("location", randomRepoPath()) - .put(MockRepository.Plugin.USERNAME_SETTING.getKey(), "notsecretusername") - .put(MockRepository.Plugin.PASSWORD_SETTING.getKey(), "verysecretpassword")); + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", randomRepoPath()) + .put(MockRepository.Plugin.USERNAME_SETTING.getKey(), "notsecretusername") + .put(MockRepository.Plugin.PASSWORD_SETTING.getKey(), "verysecretpassword") + ); NodeClient nodeClient = internalCluster().getInstance(NodeClient.class); RestGetRepositoriesAction getRepoAction = new RestGetRepositoriesAction(internalCluster().getInstance(SettingsFilter.class)); @@ -437,8 +508,10 @@ public void sendResponse(RestResponse response) { throw getRepoError.get(); } - RestClusterStateAction clusterStateAction = new RestClusterStateAction(internalCluster().getInstance(SettingsFilter.class), - internalCluster().getInstance(ThreadPool.class)); + RestClusterStateAction clusterStateAction = new RestClusterStateAction( + internalCluster().getInstance(SettingsFilter.class), + internalCluster().getInstance(ThreadPool.class) + ); RestRequest clusterStateRequest = new FakeRestRequest(); final CountDownLatch clusterStateLatch = new CountDownLatch(1); final AtomicReference clusterStateError = new AtomicReference<>(); @@ -479,8 +552,12 @@ public void testMasterShutdownDuringSnapshot() throws Exception { final int numberOfShards = getNumShards("test-idx").numPrimaries; logger.info("number of shards: {}", numberOfShards); - dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false) - .setIndices("test-idx").get(); + dataNodeClient().admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIndices("test-idx") + .get(); logger.info("--> stopping master node"); internalCluster().stopCurrentMasterNode(); @@ -516,8 +593,12 @@ public void testMasterAndDataShutdownDuringSnapshot() throws Exception { blockMasterFromFinalizingSnapshotOnSnapFile("test-repo"); final String dataNode = blockNodeWithIndex("test-repo", "test-idx"); - dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false) - .setIndices("test-idx").get(); + dataNodeClient().admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIndices("test-idx") + .get(); stopNode(dataNode); internalCluster().stopCurrentMasterNode(); @@ -558,8 +639,13 @@ public void testRestoreShrinkIndex() throws Exception { indexRandomDocs(sourceIdx, randomIntBetween(10, 100)); logger.info("--> shrink the index"); - assertAcked(client().admin().indices().prepareUpdateSettings(sourceIdx) - .setSettings(Settings.builder().put("index.blocks.write", true)).get()); + assertAcked( + client().admin() + .indices() + .prepareUpdateSettings(sourceIdx) + .setSettings(Settings.builder().put("index.blocks.write", true)) + .get() + ); assertAcked(client().admin().indices().prepareResizeIndex(sourceIdx, shrunkIdx).get()); logger.info("--> snapshot the shrunk index"); @@ -580,11 +666,11 @@ public void testRestoreShrinkIndex() throws Exception { clusterAdmin().prepareHealth().setTimeout("30s").setWaitForNodes("2"); logger.info("--> restore the shrunk index and ensure all shards are allocated"); - RestoreSnapshotResponse restoreResponse = clusterAdmin() - .prepareRestoreSnapshot(repo, snapshot).setWaitForCompletion(true) - .setIndices(shrunkIdx).get(); - assertEquals(restoreResponse.getRestoreInfo().totalShards(), - restoreResponse.getRestoreInfo().successfulShards()); + RestoreSnapshotResponse restoreResponse = clusterAdmin().prepareRestoreSnapshot(repo, snapshot) + .setWaitForCompletion(true) + .setIndices(shrunkIdx) + .get(); + assertEquals(restoreResponse.getRestoreInfo().totalShards(), restoreResponse.getRestoreInfo().successfulShards()); ensureYellow(); } @@ -595,9 +681,11 @@ public void testSnapshotWithDateMath() { final String snapshotName = ""; logger.info("--> creating repository"); - assertAcked(clusterAdmin().preparePutRepository(repo).setType("fs") - .setSettings(Settings.builder().put("location", randomRepoPath()) - .put("compress", randomBoolean()))); + assertAcked( + clusterAdmin().preparePutRepository(repo) + .setType("fs") + .setSettings(Settings.builder().put("location", randomRepoPath()).put("compress", randomBoolean())) + ); final String expression1 = nameExpressionResolver.resolveDateMathExpression(snapshotName); logger.info("--> creating date math snapshot"); @@ -631,9 +719,7 @@ public void testSnapshotTotalAndIncrementalSizes() throws Exception { createRepository(repositoryName, "fs", repoPath); createFullSnapshot(repositoryName, snapshot0); - SnapshotsStatusResponse response = clusterAdmin().prepareSnapshotStatus(repositoryName) - .setSnapshots(snapshot0) - .get(); + SnapshotsStatusResponse response = clusterAdmin().prepareSnapshotStatus(repositoryName).setSnapshots(snapshot0).get(); List snapshots = response.getSnapshots(); @@ -666,7 +752,7 @@ public void testSnapshotTotalAndIncrementalSizes() throws Exception { // total size has to grow and has to be equal to files on fs createFullSnapshot(repositoryName, snapshot1); - // drop 1st one to avoid miscalculation as snapshot reuses some files of prev snapshot + // drop 1st one to avoid miscalculation as snapshot reuses some files of prev snapshot assertAcked(startDeleteSnapshot(repositoryName, snapshot0).get()); response = clusterAdmin().prepareSnapshotStatus(repositoryName).setSnapshots(snapshot1).get(); @@ -768,23 +854,40 @@ public void testDataNodeRestartWithBusyMasterDuringSnapshot() throws Exception { logger.info("--> snapshot"); ServiceDisruptionScheme disruption = new BusyMasterServiceDisruption(random(), Priority.HIGH); setDisruptionScheme(disruption); - client(internalCluster().getMasterName()).admin().cluster() - .prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); + client(internalCluster().getMasterName()).admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIndices("test-idx") + .get(); disruption.startDisrupting(); logger.info("--> restarting data node, which should cause primary shards to be failed"); internalCluster().restartNode(dataNode, InternalTestCluster.EMPTY_CALLBACK); logger.info("--> wait for shard snapshots to show as failed"); - assertBusy(() -> assertThat( - clusterAdmin().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get().getSnapshots() - .get(0).getShardsStats().getFailedShards(), greaterThanOrEqualTo(1)), 60L, TimeUnit.SECONDS); + assertBusy( + () -> assertThat( + clusterAdmin().prepareSnapshotStatus("test-repo") + .setSnapshots("test-snap") + .get() + .getSnapshots() + .get(0) + .getShardsStats() + .getFailedShards(), + greaterThanOrEqualTo(1) + ), + 60L, + TimeUnit.SECONDS + ); unblockNode("test-repo", dataNode); disruption.stopDisrupting(); // check that snapshot completes assertBusy(() -> { - GetSnapshotsResponse snapshotsStatusResponse = clusterAdmin() - .prepareGetSnapshots("test-repo").setSnapshots("test-snap").setIgnoreUnavailable(true).get(); + GetSnapshotsResponse snapshotsStatusResponse = clusterAdmin().prepareGetSnapshots("test-repo") + .setSnapshots("test-snap") + .setIgnoreUnavailable(true) + .get(); assertEquals(1, snapshotsStatusResponse.getSnapshots("test-repo").size()); SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots("test-repo").get(0); assertTrue(snapshotInfo.state().toString(), snapshotInfo.state().completed()); @@ -805,23 +908,40 @@ public void testDataNodeRestartAfterShardSnapshotFailure() throws Exception { blockAllDataNodes("test-repo"); logger.info("--> snapshot"); - client(internalCluster().getMasterName()).admin().cluster() - .prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); + client(internalCluster().getMasterName()).admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIndices("test-idx") + .get(); logger.info("--> restarting first data node, which should cause the primary shard on it to be failed"); internalCluster().restartNode(dataNodes.get(0), InternalTestCluster.EMPTY_CALLBACK); logger.info("--> wait for shard snapshot of first primary to show as failed"); - assertBusy(() -> assertThat( - clusterAdmin().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get().getSnapshots() - .get(0).getShardsStats().getFailedShards(), is(1)), 60L, TimeUnit.SECONDS); + assertBusy( + () -> assertThat( + clusterAdmin().prepareSnapshotStatus("test-repo") + .setSnapshots("test-snap") + .get() + .getSnapshots() + .get(0) + .getShardsStats() + .getFailedShards(), + is(1) + ), + 60L, + TimeUnit.SECONDS + ); logger.info("--> restarting second data node, which should cause the primary shard on it to be failed"); internalCluster().restartNode(dataNodes.get(1), InternalTestCluster.EMPTY_CALLBACK); // check that snapshot completes with both failed shards being accounted for in the snapshot result assertBusy(() -> { - GetSnapshotsResponse snapshotsStatusResponse = clusterAdmin() - .prepareGetSnapshots("test-repo").setSnapshots("test-snap").setIgnoreUnavailable(true).get(); + GetSnapshotsResponse snapshotsStatusResponse = clusterAdmin().prepareGetSnapshots("test-repo") + .setSnapshots("test-snap") + .setIgnoreUnavailable(true) + .get(); assertEquals(1, snapshotsStatusResponse.getSnapshots("test-repo").size()); SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots("test-repo").get(0); assertTrue(snapshotInfo.state().toString(), snapshotInfo.state().completed()); @@ -850,11 +970,13 @@ public void testRetentionLeasesClearedOnRestore() throws Exception { final String leaseId = randomAlphaOfLength(randomIntBetween(1, 10)).toLowerCase(Locale.ROOT); logger.debug("--> adding retention lease with id {} to {}", leaseId, shardId); - client().execute(RetentionLeaseActions.Add.INSTANCE, new RetentionLeaseActions.AddRequest( - shardId, leaseId, RETAIN_ALL, "test")).actionGet(); + client().execute(RetentionLeaseActions.Add.INSTANCE, new RetentionLeaseActions.AddRequest(shardId, leaseId, RETAIN_ALL, "test")) + .actionGet(); final ShardStats shardStats = Arrays.stream(client().admin().indices().prepareStats(indexName).get().getShards()) - .filter(s -> s.getShardRouting().shardId().equals(shardId)).findFirst().get(); + .filter(s -> s.getShardRouting().shardId().equals(shardId)) + .findFirst() + .get(); final RetentionLeases retentionLeases = shardStats.getRetentionLeaseStats().retentionLeases(); assertTrue(shardStats + ": " + retentionLeases, retentionLeases.contains(leaseId)); @@ -878,16 +1000,20 @@ public void testRetentionLeasesClearedOnRestore() throws Exception { logger.debug("--> restore index {} from snapshot", indexName); RestoreSnapshotResponse restoreResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapshotName) - .setWaitForCompletion(true).get(); + .setWaitForCompletion(true) + .get(); assertThat(restoreResponse.getRestoreInfo().successfulShards(), equalTo(shardCount)); assertThat(restoreResponse.getRestoreInfo().failedShards(), equalTo(0)); ensureGreen(); assertDocCount(indexName, snapshotDocCount); - final RetentionLeases restoredRetentionLeases = Arrays.stream(client().admin().indices().prepareStats(indexName).get() - .getShards()).filter(s -> s.getShardRouting().shardId().equals(shardId)).findFirst().get() - .getRetentionLeaseStats().retentionLeases(); + final RetentionLeases restoredRetentionLeases = Arrays.stream(client().admin().indices().prepareStats(indexName).get().getShards()) + .filter(s -> s.getShardRouting().shardId().equals(shardId)) + .findFirst() + .get() + .getRetentionLeaseStats() + .retentionLeases(); assertFalse(restoredRetentionLeases.toString() + " has no " + leaseId, restoredRetentionLeases.contains(leaseId)); } @@ -912,8 +1038,13 @@ public void testAbortWaitsOnDataNode() throws Exception { final TransportService transportService = internalCluster().getInstance(TransportService.class, otherDataNode); transportService.addMessageListener(new TransportMessageListener() { @Override - public void onRequestSent(DiscoveryNode node, long requestId, String action, TransportRequest request, - TransportRequestOptions finalOptions) { + public void onRequestSent( + DiscoveryNode node, + long requestId, + String action, + TransportRequest request, + TransportRequestOptions finalOptions + ) { if (blocked.get() && action.equals(SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME)) { throw new AssertionError("Node had no assigned shard snapshots so it shouldn't send out shard state updates"); } @@ -923,8 +1054,13 @@ public void onRequestSent(DiscoveryNode node, long requestId, String action, Tra logger.info("--> abort snapshot"); final ActionFuture deleteResponse = startDeleteSnapshot(repoName, snapshotName); - awaitClusterState(otherDataNode, state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) - .entries().stream().anyMatch(entry -> entry.state() == SnapshotsInProgress.State.ABORTED)); + awaitClusterState( + otherDataNode, + state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .entries() + .stream() + .anyMatch(entry -> entry.state() == SnapshotsInProgress.State.ABORTED) + ); assertFalse("delete should not be able to finish until data node is unblocked", deleteResponse.isDone()); blocked.set(false); @@ -960,12 +1096,13 @@ public void testSnapshotDeleteRelocatingPrimaryIndex() throws Exception { // Drop all file chunk requests so that below relocation takes forever and we're guaranteed to run the snapshot in parallel to it for (String nodeName : dataNodes) { ((MockTransportService) internalCluster().getInstance(TransportService.class, nodeName)).addSendBehavior( - (connection, requestId, action, request, options) -> { - if (PeerRecoveryTargetService.Actions.FILE_CHUNK.equals(action)) { - return; - } - connection.sendRequest(requestId, action, request, options); - }); + (connection, requestId, action, request, options) -> { + if (PeerRecoveryTargetService.Actions.FILE_CHUNK.equals(action)) { + return; + } + connection.sendRequest(requestId, action, request, options); + } + ); } logger.info("--> start relocations"); @@ -973,13 +1110,18 @@ public void testSnapshotDeleteRelocatingPrimaryIndex() throws Exception { logger.info("--> wait for relocations to start"); - assertBusy(() -> assertThat( - clusterAdmin().prepareHealth(indexName).execute().actionGet().getRelocatingShards(), greaterThan(0)), - 1L, TimeUnit.MINUTES); + assertBusy( + () -> assertThat(clusterAdmin().prepareHealth(indexName).execute().actionGet().getRelocatingShards(), greaterThan(0)), + 1L, + TimeUnit.MINUTES + ); logger.info("--> snapshot"); clusterAdmin().prepareCreateSnapshot(repoName, "test-snap") - .setWaitForCompletion(false).setPartial(true).setIndices(indexName).get(); + .setWaitForCompletion(false) + .setPartial(true) + .setIndices(indexName) + .get(); assertAcked(client().admin().indices().prepareDelete(indexName)); @@ -1050,28 +1192,27 @@ public void testDeleteIndexDuringSnapshot() throws Exception { // at which point doneListener is resolved private void startSnapshotDeleteLoop(String repoName, String indexName, String snapshotName, ActionListener doneListener) { clusterAdmin().prepareCreateSnapshot(repoName, snapshotName) - .setWaitForCompletion(true) - .setPartial(true) - .setIndices(indexName) - .execute(new ActionListener<>() { - @Override - public void onResponse(CreateSnapshotResponse createSnapshotResponse) { - clusterAdmin().prepareDeleteSnapshot(repoName, snapshotName).execute( - ActionTestUtils.assertNoFailureListener(acknowledgedResponse -> { - assertAcked(acknowledgedResponse); - startSnapshotDeleteLoop(repoName, indexName, snapshotName, doneListener); - })); - } + .setWaitForCompletion(true) + .setPartial(true) + .setIndices(indexName) + .execute(new ActionListener<>() { + @Override + public void onResponse(CreateSnapshotResponse createSnapshotResponse) { + clusterAdmin().prepareDeleteSnapshot(repoName, snapshotName) + .execute(ActionTestUtils.assertNoFailureListener(acknowledgedResponse -> { + assertAcked(acknowledgedResponse); + startSnapshotDeleteLoop(repoName, indexName, snapshotName, doneListener); + })); + } - @Override - public void onFailure(Exception e) { - assertThat(e, instanceOf(IndexNotFoundException.class)); - doneListener.onResponse(null); - } - }); + @Override + public void onFailure(Exception e) { + assertThat(e, instanceOf(IndexNotFoundException.class)); + doneListener.onResponse(null); + } + }); } - public void testGetReposWithWildcard() { internalCluster().startMasterOnlyNode(); List repositoryMetadata = client().admin().cluster().prepareGetRepositories("*").get().repositories(); @@ -1102,7 +1243,7 @@ private static List findRepoMetaBlobs(Path repoPath) throws IOException { private List scanSnapshotFolder(Path repoPath) throws IOException { List files = new ArrayList<>(); forEachFileRecursively(repoPath.resolve("indices"), ((file, basicFileAttributes) -> { - if (file.getFileName().toString().startsWith("__")){ + if (file.getFileName().toString().startsWith("__")) { files.add(file); } })); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java index 2e9968361c358..95da673c5c40c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java @@ -71,35 +71,39 @@ public void testResetSystemIndices() throws Exception { // call the reset API ResetFeatureStateResponse apiResponse = client().execute(ResetFeatureStateAction.INSTANCE, new ResetFeatureStateRequest()).get(); - assertThat(apiResponse.getFeatureStateResetStatuses(), containsInAnyOrder( - ResetFeatureStateResponse.ResetFeatureStateStatus.success("SystemIndexTestPlugin"), - ResetFeatureStateResponse.ResetFeatureStateStatus.success("SecondSystemIndexTestPlugin"), - ResetFeatureStateResponse.ResetFeatureStateStatus.success("EvilSystemIndexTestPlugin"), - ResetFeatureStateResponse.ResetFeatureStateStatus.success("tasks") - )); + assertThat( + apiResponse.getFeatureStateResetStatuses(), + containsInAnyOrder( + ResetFeatureStateResponse.ResetFeatureStateStatus.success("SystemIndexTestPlugin"), + ResetFeatureStateResponse.ResetFeatureStateStatus.success("SecondSystemIndexTestPlugin"), + ResetFeatureStateResponse.ResetFeatureStateStatus.success("EvilSystemIndexTestPlugin"), + ResetFeatureStateResponse.ResetFeatureStateStatus.success("tasks") + ) + ); // verify that both indices are gone - Exception e1 = expectThrows(IndexNotFoundException.class, () -> client().admin().indices().prepareGetIndex() - .addIndices(systemIndex1) - .get()); + Exception e1 = expectThrows( + IndexNotFoundException.class, + () -> client().admin().indices().prepareGetIndex().addIndices(systemIndex1).get() + ); assertThat(e1.getMessage(), containsString("no such index")); - Exception e2 = expectThrows(IndexNotFoundException.class, () -> client().admin().indices().prepareGetIndex() - .addIndices(associatedIndex) - .get()); + Exception e2 = expectThrows( + IndexNotFoundException.class, + () -> client().admin().indices().prepareGetIndex().addIndices(associatedIndex).get() + ); assertThat(e2.getMessage(), containsString("no such index")); - Exception e3 = expectThrows(IndexNotFoundException.class, () -> client().admin().indices().prepareGetIndex() - .addIndices(systemIndex2) - .get()); + Exception e3 = expectThrows( + IndexNotFoundException.class, + () -> client().admin().indices().prepareGetIndex().addIndices(systemIndex2).get() + ); assertThat(e3.getMessage(), containsString("no such index")); - GetIndexResponse response = client().admin().indices().prepareGetIndex() - .addIndices("my_index") - .get(); + GetIndexResponse response = client().admin().indices().prepareGetIndex().addIndices("my_index").get(); assertThat(response.getIndices(), arrayContaining("my_index")); } @@ -111,10 +115,13 @@ public void testResetSystemIndices() throws Exception { public void testFeatureResetFailure() throws Exception { try { EvilSystemIndexTestPlugin.setBeEvil(true); - ResetFeatureStateResponse resetFeatureStateResponse = client().execute(ResetFeatureStateAction.INSTANCE, - new ResetFeatureStateRequest()).get(); + ResetFeatureStateResponse resetFeatureStateResponse = client().execute( + ResetFeatureStateAction.INSTANCE, + new ResetFeatureStateRequest() + ).get(); - List failedFeatures = resetFeatureStateResponse.getFeatureStateResetStatuses().stream() + List failedFeatures = resetFeatureStateResponse.getFeatureStateResetStatuses() + .stream() .filter(status -> status.getStatus() == ResetFeatureStateResponse.ResetFeatureStateStatus.Status.FAILURE) .peek(status -> assertThat(status.getException(), notNullValue())) .map(status -> { @@ -210,10 +217,12 @@ public static synchronized boolean isEvil() { public void cleanUpFeature( ClusterService clusterService, Client client, - ActionListener listener) { + ActionListener listener + ) { if (isEvil()) { - listener.onResponse(ResetFeatureStateResponse.ResetFeatureStateStatus.failure(getFeatureName(), - new ElasticsearchException("problem!"))); + listener.onResponse( + ResetFeatureStateResponse.ResetFeatureStateStatus.failure(getFeatureName(), new ElasticsearchException("problem!")) + ); } else { listener.onResponse(ResetFeatureStateResponse.ResetFeatureStateStatus.success(getFeatureName())); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java index 349f5519903af..c333ad2cf843a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java @@ -55,12 +55,14 @@ protected Collection> nodePlugins() { public void testWhenMetadataAreLoaded() throws Exception { createIndex("docs"); - indexRandom(true, + indexRandom( + true, client().prepareIndex("docs").setId("1").setSource("rank", 1), client().prepareIndex("docs").setId("2").setSource("rank", 2), client().prepareIndex("docs").setId("3").setSource("rank", 3), client().prepareIndex("others").setSource("rank", 4), - client().prepareIndex("others").setSource("rank", 5)); + client().prepareIndex("others").setSource("rank", 5) + ); createRepository("repository", CountingMockRepositoryPlugin.TYPE); @@ -73,16 +75,23 @@ public void testWhenMetadataAreLoaded() throws Exception { assertIndexMetadataLoads("snap", "others", 0); // Getting a snapshot does not load any metadata - GetSnapshotsResponse getSnapshotsResponse = - client().admin().cluster().prepareGetSnapshots("repository").addSnapshots("snap").setVerbose(randomBoolean()).get(); + GetSnapshotsResponse getSnapshotsResponse = client().admin() + .cluster() + .prepareGetSnapshots("repository") + .addSnapshots("snap") + .setVerbose(randomBoolean()) + .get(); assertThat(getSnapshotsResponse.getSnapshots("repository"), hasSize(1)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 0); assertIndexMetadataLoads("snap", "others", 0); // Getting the status of a snapshot loads indices metadata but not global metadata - SnapshotsStatusResponse snapshotStatusResponse = - client().admin().cluster().prepareSnapshotStatus("repository").setSnapshots("snap").get(); + SnapshotsStatusResponse snapshotStatusResponse = client().admin() + .cluster() + .prepareSnapshotStatus("repository") + .setSnapshots("snap") + .get(); assertThat(snapshotStatusResponse.getSnapshots(), hasSize(1)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 1); @@ -91,9 +100,11 @@ public void testWhenMetadataAreLoaded() throws Exception { assertAcked(client().admin().indices().prepareDelete("docs", "others")); // Restoring a snapshot loads indices metadata but not the global state - RestoreSnapshotResponse restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot("repository", "snap") - .setWaitForCompletion(true) - .get(); + RestoreSnapshotResponse restoreSnapshotResponse = client().admin() + .cluster() + .prepareRestoreSnapshot("repository", "snap") + .setWaitForCompletion(true) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 2); @@ -102,10 +113,12 @@ public void testWhenMetadataAreLoaded() throws Exception { assertAcked(client().admin().indices().prepareDelete("docs")); // Restoring a snapshot with selective indices loads only required index metadata - restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot("repository", "snap") - .setIndices("docs") - .setWaitForCompletion(true) - .get(); + restoreSnapshotResponse = client().admin() + .cluster() + .prepareRestoreSnapshot("repository", "snap") + .setIndices("docs") + .setWaitForCompletion(true) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 3); @@ -114,7 +127,9 @@ public void testWhenMetadataAreLoaded() throws Exception { assertAcked(client().admin().indices().prepareDelete("docs", "others")); // Restoring a snapshot including the global state loads it with the index metadata - restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot("repository", "snap") + restoreSnapshotResponse = client().admin() + .cluster() + .prepareRestoreSnapshot("repository", "snap") .setIndices("docs", "oth*") .setRestoreGlobalState(true) .setWaitForCompletion(true) @@ -155,7 +170,7 @@ private CountingMockRepository getCountingMockRepository() { RepositoriesService repositoriesService = internalCluster().getInstance(RepositoriesService.class, master); Repository repository = repositoriesService.repository("repository"); assertThat(repository, instanceOf(CountingMockRepository.class)); - return (CountingMockRepository) repository; + return (CountingMockRepository) repository; } /** Compute a map key for the given snapshot and index names **/ @@ -169,10 +184,14 @@ public static class CountingMockRepository extends MockRepository { final Map globalMetadata = new ConcurrentHashMap<>(); final Map indicesMetadata = new ConcurrentHashMap<>(); - public CountingMockRepository(final RepositoryMetadata metadata, - final Environment environment, - final NamedXContentRegistry namedXContentRegistry, ClusterService clusterService, - BigArrays bigArrays, RecoverySettings recoverySettings) { + public CountingMockRepository( + final RepositoryMetadata metadata, + final Environment environment, + final NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings + ) { super(metadata, environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings); } @@ -183,8 +202,8 @@ public Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) { } @Override - public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, - IndexId indexId) throws IOException { + public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, IndexId indexId) + throws IOException { indicesMetadata.computeIfAbsent(key(snapshotId.getName(), indexId.getName()), (s) -> new AtomicInteger(0)).incrementAndGet(); return super.getSnapshotIndexMetaData(PlainActionFuture.get(this::getRepositoryData), snapshotId, indexId); } @@ -196,11 +215,17 @@ public static class CountingMockRepositoryPlugin extends MockRepository.Plugin { public static final String TYPE = "countingmock"; @Override - public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry, - ClusterService clusterService, BigArrays bigArrays, - RecoverySettings recoverySettings) { - return Collections.singletonMap(TYPE, - metadata -> new CountingMockRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings)); + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings + ) { + return Collections.singletonMap( + TYPE, + metadata -> new CountingMockRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) + ); } } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MultiClusterRepoAccessIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MultiClusterRepoAccessIT.java index 9d96c665ff8a5..9d1e342822d51 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MultiClusterRepoAccessIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MultiClusterRepoAccessIT.java @@ -42,22 +42,41 @@ public class MultiClusterRepoAccessIT extends AbstractSnapshotIntegTestCase { @Before public void startSecondCluster() throws IOException, InterruptedException { repoPath = randomRepoPath(); - secondCluster = new InternalTestCluster(randomLong(), createTempDir(), true, true, 0, - 0, "second_cluster", new NodeConfigurationSource() { - @Override - public Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { - return Settings.builder().put(MultiClusterRepoAccessIT.this.nodeSettings(nodeOrdinal, otherSettings)) + secondCluster = new InternalTestCluster( + randomLong(), + createTempDir(), + true, + true, + 0, + 0, + "second_cluster", + new NodeConfigurationSource() { + @Override + public Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(MultiClusterRepoAccessIT.this.nodeSettings(nodeOrdinal, otherSettings)) .put(NetworkModule.TRANSPORT_TYPE_KEY, getTestTransportType()) - .put(Environment.PATH_REPO_SETTING.getKey(), repoPath).build(); - } - - @Override - public Path nodeConfigPath(int nodeOrdinal) { - return null; - } - }, 0, "leader", Arrays.asList(ESIntegTestCase.TestSeedPlugin.class, - MockHttpTransport.TestPlugin.class, MockTransportService.TestPlugin.class, - MockNioTransportPlugin.class, InternalSettingsPlugin.class, MockRepository.Plugin.class), Function.identity()); + .put(Environment.PATH_REPO_SETTING.getKey(), repoPath) + .build(); + } + + @Override + public Path nodeConfigPath(int nodeOrdinal) { + return null; + } + }, + 0, + "leader", + Arrays.asList( + ESIntegTestCase.TestSeedPlugin.class, + MockHttpTransport.TestPlugin.class, + MockTransportService.TestPlugin.class, + MockNioTransportPlugin.class, + InternalSettingsPlugin.class, + MockRepository.Plugin.class + ), + Function.identity() + ); secondCluster.beforeTest(random()); } @@ -83,21 +102,38 @@ public void testConcurrentDeleteFromOtherCluster() throws InterruptedException { createIndexWithRandomDocs("test-idx-3", randomIntBetween(1, 100)); createFullSnapshot(repoNameOnFirstCluster, "snap-3"); - secondCluster.client().admin().cluster().preparePutRepository(repoNameOnSecondCluster).setType("fs") - .setSettings(Settings.builder().put("location", repoPath)).get(); + secondCluster.client() + .admin() + .cluster() + .preparePutRepository(repoNameOnSecondCluster) + .setType("fs") + .setSettings(Settings.builder().put("location", repoPath)) + .get(); secondCluster.client().admin().cluster().prepareDeleteSnapshot(repoNameOnSecondCluster, "snap-1").get(); secondCluster.client().admin().cluster().prepareDeleteSnapshot(repoNameOnSecondCluster, "snap-2").get(); - final SnapshotException sne = expectThrows(SnapshotException.class, () -> - client().admin().cluster().prepareCreateSnapshot(repoNameOnFirstCluster, "snap-4").setWaitForCompletion(true) - .execute().actionGet()); + final SnapshotException sne = expectThrows( + SnapshotException.class, + () -> client().admin() + .cluster() + .prepareCreateSnapshot(repoNameOnFirstCluster, "snap-4") + .setWaitForCompletion(true) + .execute() + .actionGet() + ); assertThat(sne.getMessage(), containsString("failed to update snapshot in repository")); final RepositoryException cause = (RepositoryException) sne.getCause(); - assertThat(cause.getMessage(), containsString("[" + repoNameOnFirstCluster + - "] concurrent modification of the index-N file, expected current generation [2] but it was not found in the repository." - + " The last cluster to write to this repository was [" - + secondCluster.client().admin().cluster().prepareState().get().getState().metadata().clusterUUID() - + "] at generation [4].")); + assertThat( + cause.getMessage(), + containsString( + "[" + + repoNameOnFirstCluster + + "] concurrent modification of the index-N file, expected current generation [2] but it was not found in the repository." + + " The last cluster to write to this repository was [" + + secondCluster.client().admin().cluster().prepareState().get().getState().metadata().clusterUUID() + + "] at generation [4]." + ) + ); assertAcked(client().admin().cluster().prepareDeleteRepository(repoNameOnFirstCluster).get()); createRepository(repoNameOnFirstCluster, "fs", repoPath); createFullSnapshot(repoNameOnFirstCluster, "snap-5"); @@ -111,28 +147,73 @@ public void testConcurrentWipeAndRecreateFromOtherCluster() throws InterruptedEx createIndexWithRandomDocs("test-idx-1", randomIntBetween(1, 100)); createFullSnapshot(repoName, "snap-1"); - final String repoUuid = client().admin().cluster().prepareGetRepositories(repoName).get().repositories() - .stream().filter(r -> r.name().equals(repoName)).findFirst().orElseThrow().uuid(); + final String repoUuid = client().admin() + .cluster() + .prepareGetRepositories(repoName) + .get() + .repositories() + .stream() + .filter(r -> r.name().equals(repoName)) + .findFirst() + .orElseThrow() + .uuid(); secondCluster.startMasterOnlyNode(); secondCluster.startDataOnlyNode(); - assertAcked(secondCluster.client().admin().cluster().preparePutRepository(repoName) + assertAcked( + secondCluster.client() + .admin() + .cluster() + .preparePutRepository(repoName) .setType("fs") - .setSettings(Settings.builder().put("location", repoPath).put(READONLY_SETTING_KEY, true))); - assertThat(secondCluster.client().admin().cluster().prepareGetRepositories(repoName).get().repositories() - .stream().filter(r -> r.name().equals(repoName)).findFirst().orElseThrow().uuid(), equalTo(repoUuid)); + .setSettings(Settings.builder().put("location", repoPath).put(READONLY_SETTING_KEY, true)) + ); + assertThat( + secondCluster.client() + .admin() + .cluster() + .prepareGetRepositories(repoName) + .get() + .repositories() + .stream() + .filter(r -> r.name().equals(repoName)) + .findFirst() + .orElseThrow() + .uuid(), + equalTo(repoUuid) + ); assertAcked(client().admin().cluster().prepareDeleteRepository(repoName)); IOUtils.rm(internalCluster().getCurrentMasterNodeInstance(Environment.class).resolveRepoFile(repoPath.toString())); createRepository(repoName, "fs", repoPath); createFullSnapshot(repoName, "snap-1"); - final String newRepoUuid = client().admin().cluster().prepareGetRepositories(repoName).get().repositories() - .stream().filter(r -> r.name().equals(repoName)).findFirst().orElseThrow().uuid(); + final String newRepoUuid = client().admin() + .cluster() + .prepareGetRepositories(repoName) + .get() + .repositories() + .stream() + .filter(r -> r.name().equals(repoName)) + .findFirst() + .orElseThrow() + .uuid(); assertThat(newRepoUuid, not(equalTo((repoUuid)))); secondCluster.client().admin().cluster().prepareGetSnapshots(repoName).get(); // force another read of the repo data - assertThat(secondCluster.client().admin().cluster().prepareGetRepositories(repoName).get().repositories() - .stream().filter(r -> r.name().equals(repoName)).findFirst().orElseThrow().uuid(), equalTo(newRepoUuid)); + assertThat( + secondCluster.client() + .admin() + .cluster() + .prepareGetRepositories(repoName) + .get() + .repositories() + .stream() + .filter(r -> r.name().equals(repoName)) + .findFirst() + .orElseThrow() + .uuid(), + equalTo(newRepoUuid) + ); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoriesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoriesIT.java index d33f8e14defe0..b6ccc015cf499 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoriesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoriesIT.java @@ -73,8 +73,10 @@ public void testRepositoryCreation() throws Exception { assertThat(repositoriesMetadata.repository("test-repo-2").type(), equalTo("fs")); logger.info("--> check that both repositories can be retrieved by getRepositories query"); - GetRepositoriesResponse repositoriesResponse = client.admin().cluster() - .prepareGetRepositories(randomFrom("_all", "*", "test-repo-*")).get(); + GetRepositoriesResponse repositoriesResponse = client.admin() + .cluster() + .prepareGetRepositories(randomFrom("_all", "*", "test-repo-*")) + .get(); assertThat(repositoriesResponse.repositories().size(), equalTo(2)); assertThat(findRepository(repositoriesResponse.repositories(), "test-repo-1"), notNullValue()); assertThat(findRepository(repositoriesResponse.repositories(), "test-repo-2"), notNullValue()); @@ -82,11 +84,15 @@ public void testRepositoryCreation() throws Exception { logger.info("--> check that trying to create a repository with the same settings repeatedly does not update cluster state"); String beforeStateUuid = clusterStateResponse.getState().stateUUID(); assertThat( - client.admin().cluster().preparePutRepository("test-repo-1") - .setType("fs").setSettings(Settings.builder() - .put("location", location) - ).get().isAcknowledged(), - equalTo(true)); + client.admin() + .cluster() + .preparePutRepository("test-repo-1") + .setType("fs") + .setSettings(Settings.builder().put("location", location)) + .get() + .isAcknowledged(), + equalTo(true) + ); assertEquals(beforeStateUuid, client.admin().cluster().prepareState().clear().get().getState().stateUUID()); logger.info("--> delete repository test-repo-1"); @@ -125,40 +131,57 @@ public void testMisconfiguredRepository() { Path invalidRepoPath = createTempDir().toAbsolutePath(); String location = invalidRepoPath.toString(); try { - client().admin().cluster().preparePutRepository("test-repo") - .setType("fs").setSettings(Settings.builder().put("location", location)) - .get(); + client().admin() + .cluster() + .preparePutRepository("test-repo") + .setType("fs") + .setSettings(Settings.builder().put("location", location)) + .get(); fail("Shouldn't be here"); } catch (RepositoryException ex) { assertThat( ex.getCause().getMessage(), - containsString("location [" + location + "] doesn't match any of the locations specified by path.repo")); + containsString("location [" + location + "] doesn't match any of the locations specified by path.repo") + ); } } public void testRepositoryAckTimeout() { logger.info("--> creating repository test-repo-1 with 0s timeout - shouldn't ack"); - AcknowledgedResponse putRepositoryResponse = client().admin().cluster().preparePutRepository("test-repo-1") - .setType("fs").setSettings(Settings.builder() - .put("location", randomRepoPath()) - .put("compress", randomBoolean()) - .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES) - ) - .setTimeout("0s").get(); + AcknowledgedResponse putRepositoryResponse = client().admin() + .cluster() + .preparePutRepository("test-repo-1") + .setType("fs") + .setSettings( + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES) + ) + .setTimeout("0s") + .get(); assertThat(putRepositoryResponse.isAcknowledged(), equalTo(false)); logger.info("--> creating repository test-repo-2 with standard timeout - should ack"); - putRepositoryResponse = client().admin().cluster().preparePutRepository("test-repo-2") - .setType("fs").setSettings(Settings.builder() - .put("location", randomRepoPath()) - .put("compress", randomBoolean()) - .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES) - ).get(); + putRepositoryResponse = client().admin() + .cluster() + .preparePutRepository("test-repo-2") + .setType("fs") + .setSettings( + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES) + ) + .get(); assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true)); logger.info("--> deleting repository test-repo-2 with 0s timeout - shouldn't ack"); - AcknowledgedResponse deleteRepositoryResponse = client().admin().cluster().prepareDeleteRepository("test-repo-2") - .setTimeout("0s").get(); + AcknowledgedResponse deleteRepositoryResponse = client().admin() + .cluster() + .prepareDeleteRepository("test-repo-2") + .setTimeout("0s") + .get(); assertThat(deleteRepositoryResponse.isAcknowledged(), equalTo(false)); logger.info("--> deleting repository test-repo-1 with standard timeout - should ack"); @@ -171,31 +194,30 @@ public void testRepositoryVerification() { Client client = client(); - Settings settings = Settings.builder() - .put("location", randomRepoPath()) - .put("random_control_io_exception_rate", 1.0).build(); - Settings readonlySettings = Settings.builder().put(settings) - .put(READONLY_SETTING_KEY, true).build(); + Settings settings = Settings.builder().put("location", randomRepoPath()).put("random_control_io_exception_rate", 1.0).build(); + Settings readonlySettings = Settings.builder().put(settings).put(READONLY_SETTING_KEY, true).build(); logger.info("--> creating repository that cannot write any files - should fail"); - assertRequestBuilderThrows(client.admin().cluster().preparePutRepository("test-repo-1") - .setType("mock").setSettings(settings), - RepositoryVerificationException.class); + assertRequestBuilderThrows( + client.admin().cluster().preparePutRepository("test-repo-1").setType("mock").setSettings(settings), + RepositoryVerificationException.class + ); logger.info("--> creating read-only repository that cannot read any files - should fail"); - assertRequestBuilderThrows(client.admin().cluster().preparePutRepository("test-repo-2") - .setType("mock").setSettings(readonlySettings), - RepositoryVerificationException.class); + assertRequestBuilderThrows( + client.admin().cluster().preparePutRepository("test-repo-2").setType("mock").setSettings(readonlySettings), + RepositoryVerificationException.class + ); logger.info("--> creating repository that cannot write any files, but suppress verification - should be acked"); - assertAcked(client.admin().cluster().preparePutRepository("test-repo-1") - .setType("mock").setSettings(settings).setVerify(false)); + assertAcked(client.admin().cluster().preparePutRepository("test-repo-1").setType("mock").setSettings(settings).setVerify(false)); logger.info("--> verifying repository"); assertRequestBuilderThrows(client.admin().cluster().prepareVerifyRepository("test-repo-1"), RepositoryVerificationException.class); logger.info("--> creating read-only repository that cannot read any files, but suppress verification - should be acked"); - assertAcked(client.admin().cluster().preparePutRepository("test-repo-2") - .setType("mock").setSettings(readonlySettings).setVerify(false)); + assertAcked( + client.admin().cluster().preparePutRepository("test-repo-2").setType("mock").setSettings(readonlySettings).setVerify(false) + ); logger.info("--> verifying repository"); assertRequestBuilderThrows(client.admin().cluster().prepareVerifyRepository("test-repo-2"), RepositoryVerificationException.class); @@ -204,12 +226,12 @@ public void testRepositoryVerification() { logger.info("--> creating repository"); try { - client.admin().cluster().preparePutRepository("test-repo-1") - .setType("mock") - .setSettings(Settings.builder() - .put("location", location) - .put("localize_location", true) - ).get(); + client.admin() + .cluster() + .preparePutRepository("test-repo-1") + .setType("mock") + .setSettings(Settings.builder().put("location", location).put("localize_location", true)) + .get(); fail("RepositoryVerificationException wasn't generated"); } catch (RepositoryVerificationException ex) { assertThat(ExceptionsHelper.stackTrace(ex), containsString("is not shared")); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java index 3fd1ff0ac8927..f15f6a1149f98 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java @@ -44,12 +44,22 @@ protected Collection> nodePlugins() { public void testFilteredRepoMetadataIsUsed() { final String masterName = internalCluster().getMasterName(); final String repoName = "test-repo"; - assertAcked(client().admin().cluster().preparePutRepository(repoName).setType(MetadataFilteringPlugin.TYPE).setSettings( - Settings.builder().put("location", randomRepoPath()) - .put(MetadataFilteringPlugin.MASTER_SETTING_VALUE, masterName))); + assertAcked( + client().admin() + .cluster() + .preparePutRepository(repoName) + .setType(MetadataFilteringPlugin.TYPE) + .setSettings( + Settings.builder().put("location", randomRepoPath()).put(MetadataFilteringPlugin.MASTER_SETTING_VALUE, masterName) + ) + ); createIndex("test-idx"); - final SnapshotInfo snapshotInfo = client().admin().cluster().prepareCreateSnapshot(repoName, "test-snap") - .setWaitForCompletion(true).get().getSnapshotInfo(); + final SnapshotInfo snapshotInfo = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, "test-snap") + .setWaitForCompletion(true) + .get() + .getSnapshotInfo(); assertThat(snapshotInfo.userMetadata(), is(Collections.singletonMap(MetadataFilteringPlugin.MOCK_FILTERED_META, masterName))); } @@ -63,23 +73,40 @@ public static final class MetadataFilteringPlugin extends org.elasticsearch.plug private static final String TYPE = "mock_meta_filtering"; @Override - public Map getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry, - ClusterService clusterService, BigArrays bigArrays, - RecoverySettings recoverySettings) { - return Collections.singletonMap("mock_meta_filtering", metadata -> - new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) { + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings + ) { + return Collections.singletonMap( + "mock_meta_filtering", + metadata -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) { // Storing the initially expected metadata value here to verify that #filterUserMetadata is only called once on the // initial master node starting the snapshot private final String initialMetaValue = metadata.settings().get(MASTER_SETTING_VALUE); @Override - public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryStateId, - Metadata clusterMetadata, SnapshotInfo snapshotInfo, Version repositoryMetaVersion, - Function stateTransformer, - ActionListener listener) { - super.finalizeSnapshot(shardGenerations, repositoryStateId, clusterMetadata, snapshotInfo, - repositoryMetaVersion, stateTransformer, listener); + public void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ) { + super.finalizeSnapshot( + shardGenerations, + repositoryStateId, + clusterMetadata, + snapshotInfo, + repositoryMetaVersion, + stateTransformer, + listener + ); } @Override @@ -92,7 +119,8 @@ public void snapshotShard(SnapshotShardContext context) { public Map adaptUserMetadata(Map userMetadata) { return Collections.singletonMap(MOCK_FILTERED_META, clusterService.getNodeName()); } - }); + } + ); } } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java index cef330432f74b..1438d6e34cf97 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java @@ -84,16 +84,20 @@ public void testParallelRestoreOperations() { createSnapshot(repoName, snapshotName1, Collections.singletonList(indexName1)); createSnapshot(repoName, snapshotName2, Collections.singletonList(indexName2)); - RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName1) - .setWaitForCompletion(false) - .setRenamePattern(indexName1) - .setRenameReplacement(restoredIndexName1) - .get(); - RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName2) - .setWaitForCompletion(false) - .setRenamePattern(indexName2) - .setRenameReplacement(restoredIndexName2) - .get(); + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(repoName, snapshotName1) + .setWaitForCompletion(false) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(repoName, snapshotName2) + .setWaitForCompletion(false) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); assertThat(restoreSnapshotResponse1.status(), equalTo(RestStatus.ACCEPTED)); assertThat(restoreSnapshotResponse2.status(), equalTo(RestStatus.ACCEPTED)); ensureGreen(restoredIndexName1, restoredIndexName2); @@ -124,21 +128,23 @@ public void testParallelRestoreOperationsFromSingleSnapshot() throws Exception { createSnapshot(repoName, snapshotName, Arrays.asList(indexName1, indexName2)); - ActionFuture restoreSnapshotResponse1 = client.admin().cluster() - .prepareRestoreSnapshot(repoName, snapshotName) - .setIndices(indexName1) - .setRenamePattern(indexName1) - .setRenameReplacement(restoredIndexName1) - .execute(); + ActionFuture restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(repoName, snapshotName) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .execute(); boolean sameSourceIndex = randomBoolean(); - ActionFuture restoreSnapshotResponse2 = client.admin().cluster() - .prepareRestoreSnapshot(repoName, snapshotName) - .setIndices(sameSourceIndex ? indexName1 : indexName2) - .setRenamePattern(sameSourceIndex ? indexName1 : indexName2) - .setRenameReplacement(restoredIndexName2) - .execute(); + ActionFuture restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(repoName, snapshotName) + .setIndices(sameSourceIndex ? indexName1 : indexName2) + .setRenamePattern(sameSourceIndex ? indexName1 : indexName2) + .setRenameReplacement(restoredIndexName2) + .execute(); assertThat(restoreSnapshotResponse1.get().status(), equalTo(RestStatus.ACCEPTED)); assertThat(restoreSnapshotResponse2.get().status(), equalTo(RestStatus.ACCEPTED)); ensureGreen(restoredIndexName1, restoredIndexName2); @@ -159,12 +165,19 @@ public void testRestoreIncreasesPrimaryTerms() { } } - final IndexMetadata indexMetadata = clusterAdmin().prepareState().clear().setIndices(indexName) - .setMetadata(true).get().getState().metadata().index(indexName); + final IndexMetadata indexMetadata = clusterAdmin().prepareState() + .clear() + .setIndices(indexName) + .setMetadata(true) + .get() + .getState() + .metadata() + .index(indexName); assertThat(indexMetadata.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID), nullValue()); final int numPrimaries = getNumShards(indexName).numPrimaries; final Map primaryTerms = IntStream.range(0, numPrimaries) - .boxed().collect(Collectors.toMap(shardId -> shardId, indexMetadata::primaryTerm)); + .boxed() + .collect(Collectors.toMap(shardId -> shardId, indexMetadata::primaryTerm)); createRepository("test-repo", "fs"); createSnapshot("test-repo", "test-snap", Collections.singletonList(indexName)); @@ -172,12 +185,19 @@ public void testRestoreIncreasesPrimaryTerms() { assertAcked(client().admin().indices().prepareClose(indexName)); final RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).get(); + .setWaitForCompletion(true) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(numPrimaries)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - final IndexMetadata restoredIndexMetadata = clusterAdmin().prepareState().clear().setIndices(indexName) - .setMetadata(true).get().getState().metadata().index(indexName); + final IndexMetadata restoredIndexMetadata = clusterAdmin().prepareState() + .clear() + .setIndices(indexName) + .setMetadata(true) + .get() + .getState() + .metadata() + .index(indexName); for (int shardId = 0; shardId < numPrimaries; shardId++) { assertThat(restoredIndexMetadata.primaryTerm(shardId), greaterThan(primaryTerms.get(shardId))); } @@ -188,8 +208,16 @@ public void testRestoreWithDifferentMappingsAndSettings() throws Exception { createRepository("test-repo", "fs"); logger.info("--> create index with baz field"); - assertAcked(prepareCreate("test-idx", 2, Settings.builder() - .put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 10, TimeUnit.SECONDS))); + assertAcked( + prepareCreate( + "test-idx", + 2, + Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) + .put("refresh_interval", 10, TimeUnit.SECONDS) + ) + ); NumShards numShards = getNumShards("test-idx"); @@ -200,9 +228,16 @@ public void testRestoreWithDifferentMappingsAndSettings() throws Exception { logger.info("--> delete the index and recreate it with foo field"); cluster().wipeIndices("test-idx"); - assertAcked(prepareCreate("test-idx", 2, Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) - .put("refresh_interval", 5, TimeUnit.SECONDS))); + assertAcked( + prepareCreate( + "test-idx", + 2, + Settings.builder() + .put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries) + .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) + .put("refresh_interval", 5, TimeUnit.SECONDS) + ) + ); assertAcked(client().admin().indices().preparePutMapping("test-idx").setSource("foo", "type=text")); ensureGreen(); @@ -211,12 +246,13 @@ public void testRestoreWithDifferentMappingsAndSettings() throws Exception { logger.info("--> restore all indices from the snapshot"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); logger.info("--> assert that old mapping is restored"); - MappingMetadata mappings = clusterAdmin().prepareState().get().getState().getMetadata() - .getIndices().get("test-idx").mapping(); + MappingMetadata mappings = clusterAdmin().prepareState().get().getState().getMetadata().getIndices().get("test-idx").mapping(); assertThat(mappings.sourceAsMap().toString(), containsString("baz")); assertThat(mappings.sourceAsMap().toString(), not(containsString("foo"))); @@ -233,12 +269,16 @@ public void testRestoreAliases() throws Exception { ensureGreen(); logger.info("--> create aliases"); - assertAcked(client().admin().indices().prepareAliases() + assertAcked( + client().admin() + .indices() + .prepareAliases() .addAlias("test-idx-1", "alias-123") .addAlias("test-idx-2", "alias-123") .addAlias("test-idx-3", "alias-123") .addAlias("test-idx-1", "alias-1") - .get()); + .get() + ); assertFalse(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty()); @@ -251,10 +291,15 @@ public void testRestoreAliases() throws Exception { logger.info("--> restore snapshot with aliases"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .execute() + .actionGet(); // We don't restore any indices here - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()))); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + allOf(greaterThan(0), equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())) + ); logger.info("--> check that aliases are restored"); assertFalse(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty()); @@ -271,11 +316,17 @@ public void testRestoreAliases() throws Exception { assertTrue(client().admin().indices().prepareGetAliases("alias-1").get().getAliases().isEmpty()); logger.info("--> restore snapshot without aliases"); - restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true) - .setRestoreGlobalState(true).setIncludeAliases(false).execute().actionGet(); + restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .setIncludeAliases(false) + .execute() + .actionGet(); // We don't restore any indices here - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()))); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + allOf(greaterThan(0), equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())) + ); logger.info("--> check that aliases are not restored and existing aliases still exist"); assertTrue(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty()); @@ -287,10 +338,13 @@ public void testRestoreTemplates() throws Exception { createRepository("test-repo", "fs"); logger.info("--> creating test template"); - assertThat(client().admin().indices() + assertThat( + client().admin() + .indices() .preparePutTemplate("test-template") .setPatterns(Collections.singletonList("te*")) - .setMapping(XContentFactory.jsonBuilder() + .setMapping( + XContentFactory.jsonBuilder() .startObject() .startObject("_doc") .startObject("properties") @@ -304,8 +358,12 @@ public void testRestoreTemplates() throws Exception { .endObject() .endObject() .endObject() - .endObject()) - .get().isAcknowledged(), equalTo(true)); + .endObject() + ) + .get() + .isAcknowledged(), + equalTo(true) + ); createSnapshot("test-repo", "test-snap", Collections.emptyList()); assertThat(getSnapshot("test-repo", "test-snap").state(), equalTo(SnapshotState.SUCCESS)); @@ -317,7 +375,10 @@ public void testRestoreTemplates() throws Exception { logger.info("--> restore cluster state"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .execute() + .actionGet(); // We don't restore any indices here assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0)); @@ -334,7 +395,10 @@ public void testRenameOnRestore() throws Exception { createIndex("test-idx-1", "test-idx-2", "test-idx-3"); ensureGreen(); - assertAcked(client.admin().indices().prepareAliases() + assertAcked( + client.admin() + .indices() + .prepareAliases() .addAlias("test-idx-1", "alias-1", false) .addAlias("test-idx-2", "alias-2", false) .addAlias("test-idx-3", "alias-3", false) @@ -346,8 +410,14 @@ public void testRenameOnRestore() throws Exception { createSnapshot("test-repo", "test-snap", Arrays.asList("test-idx-1", "test-idx-2")); logger.info("--> restore indices with different names"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRenamePattern("(.+)") + .setRenameReplacement("$1-copy") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx-1-copy", 100L); @@ -357,8 +427,14 @@ public void testRenameOnRestore() throws Exception { client.admin().indices().prepareClose("test-idx-1-copy", "test-idx-2-copy").get(); logger.info("--> and try to restore these indices again"); - restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRenamePattern("(.+)") + .setRenameReplacement("$1-copy") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx-1-copy", 100L); @@ -368,8 +444,14 @@ public void testRenameOnRestore() throws Exception { assertAcked(client.admin().indices().prepareClose("test-idx-1", "test-idx-2-copy")); logger.info("--> restore indices with different names"); - restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setRenamePattern("(.+-2)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRenamePattern("(.+-2)") + .setRenameReplacement("$1-copy") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); logger.info("--> delete indices"); @@ -377,8 +459,14 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices using the same name"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setRenamePattern("(.+)") - .setRenameReplacement("same-name").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRenamePattern("(.+)") + .setRenameReplacement("same-name") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (SnapshotRestoreException ex) { // Expected @@ -386,8 +474,14 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices using the same name"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setRenamePattern("test-idx-2") - .setRenameReplacement("test-idx-1").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRenamePattern("test-idx-2") + .setRenameReplacement("test-idx-1") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (SnapshotRestoreException ex) { // Expected @@ -395,8 +489,15 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices using invalid index name"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern(".+") - .setRenameReplacement("__WRONG__").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndices("test-idx-1") + .setRenamePattern(".+") + .setRenameReplacement("__WRONG__") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (InvalidIndexNameException ex) { // Expected @@ -404,8 +505,15 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices into existing alias name"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern(".+") - .setRenameReplacement("alias-3").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndices("test-idx-1") + .setRenamePattern(".+") + .setRenameReplacement("alias-3") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (InvalidIndexNameException ex) { // Expected @@ -413,8 +521,15 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices into existing alias of itself"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern("test-idx") - .setRenameReplacement("alias").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndices("test-idx-1") + .setRenamePattern("test-idx") + .setRenameReplacement("alias") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (SnapshotRestoreException ex) { // Expected @@ -422,26 +537,42 @@ public void testRenameOnRestore() throws Exception { logger.info("--> try renaming indices into existing alias of another restored index"); try { - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1", "test-idx-2") - .setRenamePattern("test-idx-1").setRenameReplacement("alias-2").setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndices("test-idx-1", "test-idx-2") + .setRenamePattern("test-idx-1") + .setRenameReplacement("alias-2") + .setWaitForCompletion(true) + .execute() + .actionGet(); fail("Shouldn't be here"); } catch (SnapshotRestoreException ex) { // Expected } logger.info("--> try renaming indices into existing alias of itself, but don't restore aliases "); - restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setIndices("test-idx-1").setRenamePattern("test-idx").setRenameReplacement("alias") - .setWaitForCompletion(true).setIncludeAliases(false).execute().actionGet(); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndices("test-idx-1") + .setRenamePattern("test-idx") + .setRenameReplacement("alias") + .setWaitForCompletion(true) + .setIncludeAliases(false) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); } public void testDynamicRestoreThrottling() throws Exception { Client client = client(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", randomRepoPath()).put("compress", randomBoolean()) - .put("chunk_size", 100, ByteSizeUnit.BYTES)); + createRepository( + "test-repo", + "fs", + Settings.builder().put("location", randomRepoPath()).put("compress", randomBoolean()).put("chunk_size", 100, ByteSizeUnit.BYTES) + ); createIndexWithRandomDocs("test-idx", 100); createSnapshot("test-repo", "test-snap", Collections.singletonList("test-idx")); @@ -450,10 +581,16 @@ public void testDynamicRestoreThrottling() throws Exception { cluster().wipeIndices("test-idx"); logger.info("--> restore index"); - client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "100b").build()).get(); - ActionFuture restoreSnapshotResponse = client.admin().cluster() - .prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute(); + client.admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "100b").build()) + .get(); + ActionFuture restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute(); // check if throttling is active assertBusy(() -> { @@ -466,8 +603,11 @@ public void testDynamicRestoreThrottling() throws Exception { }, 30, TimeUnit.SECONDS); // run at full speed again - client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .putNull(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()).build()).get(); + client.admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()).build()) + .get(); // check that restore now completes quickly (i.e. within 20 seconds) assertThat(restoreSnapshotResponse.get(20L, TimeUnit.SECONDS).getRestoreInfo().totalShards(), greaterThan(0)); @@ -482,17 +622,21 @@ public void testChangeSettingsOnRestore() throws Exception { logger.info("--> create test index with case-preserving search analyzer"); Settings.Builder indexSettings = Settings.builder() - .put(indexSettings()) - .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) - .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s") - .put("index.analysis.analyzer.my_analyzer.type", "custom") - .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard"); + .put(indexSettings()) + .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) + .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s") + .put("index.analysis.analyzer.my_analyzer.type", "custom") + .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard"); assertAcked(prepareCreate("test-idx", 2, indexSettings)); int numberOfShards = getNumShards("test-idx").numPrimaries; - assertAcked(client().admin().indices().preparePutMapping("test-idx") - .setSource("field1", "type=text,analyzer=standard,search_analyzer=my_analyzer")); + assertAcked( + client().admin() + .indices() + .preparePutMapping("test-idx") + .setSource("field1", "type=text,analyzer=standard,search_analyzer=my_analyzer") + ); final int numdocs = randomIntBetween(10, 100); IndexRequestBuilder[] builders = new IndexRequestBuilder[numdocs]; for (int i = 0; i < builders.length; i++) { @@ -511,39 +655,50 @@ public void testChangeSettingsOnRestore() throws Exception { cluster().wipeIndices("test-idx"); Settings newIndexSettings = Settings.builder() - .put("refresh_interval", "5s") - .put("index.analysis.analyzer.my_analyzer.type", "standard") - .build(); + .put("refresh_interval", "5s") + .put("index.analysis.analyzer.my_analyzer.type", "standard") + .build(); Settings newIncorrectIndexSettings = Settings.builder() - .put(newIndexSettings) - .put(SETTING_NUMBER_OF_SHARDS, numberOfShards + 100) - .build(); + .put(newIndexSettings) + .put(SETTING_NUMBER_OF_SHARDS, numberOfShards + 100) + .build(); logger.info("--> try restoring while changing the number of shards - should fail"); - assertRequestBuilderThrows(client.admin().cluster() + assertRequestBuilderThrows( + client.admin() + .cluster() .prepareRestoreSnapshot("test-repo", "test-snap") .setIgnoreIndexSettings("index.analysis.*") .setIndexSettings(newIncorrectIndexSettings) - .setWaitForCompletion(true), SnapshotRestoreException.class); + .setWaitForCompletion(true), + SnapshotRestoreException.class + ); logger.info("--> try restoring while changing the number of replicas to a negative number - should fail"); Settings newIncorrectReplicasIndexSettings = Settings.builder() - .put(newIndexSettings) - .put(SETTING_NUMBER_OF_REPLICAS.substring(IndexMetadata.INDEX_SETTING_PREFIX.length()), randomIntBetween(-10, -1)) - .build(); - assertRequestBuilderThrows(client.admin().cluster() + .put(newIndexSettings) + .put(SETTING_NUMBER_OF_REPLICAS.substring(IndexMetadata.INDEX_SETTING_PREFIX.length()), randomIntBetween(-10, -1)) + .build(); + assertRequestBuilderThrows( + client.admin() + .cluster() .prepareRestoreSnapshot("test-repo", "test-snap") .setIgnoreIndexSettings("index.analysis.*") .setIndexSettings(newIncorrectReplicasIndexSettings) - .setWaitForCompletion(true), IllegalArgumentException.class); + .setWaitForCompletion(true), + IllegalArgumentException.class + ); logger.info("--> restore index with correct settings from the snapshot"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster() - .prepareRestoreSnapshot("test-repo", "test-snap") - .setIgnoreIndexSettings("index.analysis.*") - .setIndexSettings(newIndexSettings) - .setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIgnoreIndexSettings("index.analysis.*") + .setIndexSettings(newIndexSettings) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); logger.info("--> assert that correct settings are restored"); @@ -560,11 +715,14 @@ public void testChangeSettingsOnRestore() throws Exception { cluster().wipeIndices("test-idx"); logger.info("--> restore index with correct settings from the snapshot"); - restoreSnapshotResponse = client.admin().cluster() - .prepareRestoreSnapshot("test-repo", "test-snap") - .setIgnoreIndexSettings("*") // delete everything we can delete - .setIndexSettings(newIndexSettings) - .setWaitForCompletion(true).execute().actionGet(); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIgnoreIndexSettings("*") // delete everything we can delete + .setIndexSettings(newIndexSettings) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); logger.info("--> assert that correct settings are restored and index is still functional"); @@ -583,16 +741,20 @@ public void testRecreateBlocksOnRestore() throws Exception { createRepository("test-repo", "fs"); Settings.Builder indexSettings = Settings.builder() - .put(indexSettings()) - .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) - .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s"); + .put(indexSettings()) + .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)) + .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s"); logger.info("--> create index"); assertAcked(prepareCreate("test-idx", 2, indexSettings)); try { - List initialBlockSettings = randomSubsetOf(randomInt(3), - IndexMetadata.SETTING_BLOCKS_WRITE, IndexMetadata.SETTING_BLOCKS_METADATA, IndexMetadata.SETTING_READ_ONLY); + List initialBlockSettings = randomSubsetOf( + randomInt(3), + IndexMetadata.SETTING_BLOCKS_WRITE, + IndexMetadata.SETTING_BLOCKS_METADATA, + IndexMetadata.SETTING_READ_ONLY + ); Settings.Builder initialSettingsBuilder = Settings.builder(); for (String blockSetting : initialBlockSettings) { initialSettingsBuilder.put(blockSetting, true); @@ -615,9 +777,13 @@ public void testRecreateBlocksOnRestore() throws Exception { cluster().wipeIndices("test-idx"); logger.info("--> restore index with additional block changes"); - List changeBlockSettings = randomSubsetOf(randomInt(4), - IndexMetadata.SETTING_BLOCKS_METADATA, IndexMetadata.SETTING_BLOCKS_WRITE, - IndexMetadata.SETTING_READ_ONLY, IndexMetadata.SETTING_BLOCKS_READ); + List changeBlockSettings = randomSubsetOf( + randomInt(4), + IndexMetadata.SETTING_BLOCKS_METADATA, + IndexMetadata.SETTING_BLOCKS_WRITE, + IndexMetadata.SETTING_READ_ONLY, + IndexMetadata.SETTING_BLOCKS_READ + ); Settings.Builder changedSettingsBuilder = Settings.builder(); for (String blockSetting : changeBlockSettings) { changedSettingsBuilder.put(blockSetting, randomBoolean()); @@ -625,29 +791,37 @@ public void testRecreateBlocksOnRestore() throws Exception { Settings changedSettings = changedSettingsBuilder.build(); logger.info("--> applying changed block settings {}", changedSettings); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster() - .prepareRestoreSnapshot("test-repo", "test-snap") - .setIndexSettings(changedSettings) - .setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setIndexSettings(changedSettings) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ClusterBlocks blocks = client.admin().cluster().prepareState().clear().setBlocks(true).get().getState().blocks(); // compute current index settings (as we cannot query them if they contain SETTING_BLOCKS_METADATA) - Settings mergedSettings = Settings.builder() - .put(initialSettings) - .put(changedSettings) - .build(); + Settings mergedSettings = Settings.builder().put(initialSettings).put(changedSettings).build(); logger.info("--> merged block settings {}", mergedSettings); logger.info("--> checking consistency between settings and blocks"); - assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_METADATA, false), - is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_METADATA_BLOCK))); - assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_READ, false), - is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_BLOCK))); - assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_WRITE, false), - is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_WRITE_BLOCK))); - assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_READ_ONLY, false), - is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_ONLY_BLOCK))); + assertThat( + mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_METADATA, false), + is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_METADATA_BLOCK)) + ); + assertThat( + mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_READ, false), + is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_BLOCK)) + ); + assertThat( + mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_WRITE, false), + is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_WRITE_BLOCK)) + ); + assertThat( + mergedSettings.getAsBoolean(IndexMetadata.SETTING_READ_ONLY, false), + is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_ONLY_BLOCK)) + ); } finally { logger.info("--> cleaning up blocks"); disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_METADATA); @@ -670,11 +844,14 @@ public void testForbidDisableSoftDeletesDuringRestore() throws Exception { flush("test-index"); } createSnapshot("test-repo", "snapshot-0", Collections.singletonList("test-index")); - final SnapshotRestoreException restoreError = expectThrows(SnapshotRestoreException.class, - () -> clusterAdmin().prepareRestoreSnapshot("test-repo", "snapshot-0") - .setIndexSettings(Settings.builder().put(INDEX_SOFT_DELETES_SETTING.getKey(), false)) - .setRenamePattern("test-index").setRenameReplacement("new-index") - .get()); + final SnapshotRestoreException restoreError = expectThrows( + SnapshotRestoreException.class, + () -> clusterAdmin().prepareRestoreSnapshot("test-repo", "snapshot-0") + .setIndexSettings(Settings.builder().put(INDEX_SOFT_DELETES_SETTING.getKey(), false)) + .setRenamePattern("test-index") + .setRenameReplacement("new-index") + .get() + ); assertThat(restoreError.getMessage(), containsString("cannot disable setting [index.soft_deletes.enabled] on restore")); } @@ -684,10 +861,19 @@ public void testFailOnAncientVersion() throws Exception { createRepository(repoName, FsRepository.TYPE, repoPath); final Version oldVersion = Version.CURRENT.previousMajor().previousMajor(); final String oldSnapshot = initWithSnapshotVersion(repoName, repoPath, oldVersion); - final SnapshotRestoreException snapshotRestoreException = expectThrows(SnapshotRestoreException.class, - () -> client().admin().cluster().prepareRestoreSnapshot(repoName, oldSnapshot).execute().actionGet()); - assertThat(snapshotRestoreException.getMessage(), containsString( "the snapshot was created with Elasticsearch version [" - + oldVersion + "] which is below the current versions minimum index compatibility version [" + - Version.CURRENT.minimumIndexCompatibilityVersion() + "]")); + final SnapshotRestoreException snapshotRestoreException = expectThrows( + SnapshotRestoreException.class, + () -> client().admin().cluster().prepareRestoreSnapshot(repoName, oldSnapshot).execute().actionGet() + ); + assertThat( + snapshotRestoreException.getMessage(), + containsString( + "the snapshot was created with Elasticsearch version [" + + oldVersion + + "] which is below the current versions minimum index compatibility version [" + + Version.CURRENT.minimumIndexCompatibilityVersion() + + "]" + ) + ); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 7dbb508b60b58..29e6ba0c667d8 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -103,7 +103,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas @Override protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings)) + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order .build(); } @@ -128,11 +129,11 @@ public void testBasicWorkFlow() throws Exception { } } - final String[] indicesToSnapshot = {"test-idx-*", "-test-idx-3"}; + final String[] indicesToSnapshot = { "test-idx-*", "-test-idx-3" }; logger.info("--> capturing history UUIDs"); final Map historyUUIDs = new HashMap<>(); - for (ShardStats shardStats: client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { + for (ShardStats shardStats : client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { String historyUUID = shardStats.getCommitStats().getUserData().get(Engine.HISTORY_UUID_KEY); ShardId shardId = shardStats.getShardRouting().shardId(); if (historyUUIDs.containsKey(shardId)) { @@ -150,7 +151,9 @@ public void testBasicWorkFlow() throws Exception { createSnapshot("test-repo", "test-snap", Arrays.asList(indicesToSnapshot)); List snapshotInfos = clusterAdmin().prepareGetSnapshots("test-repo") - .setSnapshots(randomFrom("test-snap", "_all", "*", "*-snap", "test*")).get().getSnapshots("test-repo"); + .setSnapshots(randomFrom("test-snap", "_all", "*", "*-snap", "test*")) + .get() + .getSnapshots("test-repo"); assertThat(snapshotInfos.size(), equalTo(1)); SnapshotInfo snapshotInfo = snapshotInfos.get(0); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); @@ -180,7 +183,9 @@ public void testBasicWorkFlow() throws Exception { logger.info("--> restore all indices from the snapshot"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); @@ -188,10 +193,15 @@ public void testBasicWorkFlow() throws Exception { assertDocCount("test-idx-2", 100L); assertDocCount("test-idx-3", 50L); - assertNull(client().admin().indices().prepareGetSettings("test-idx-1").get().getSetting("test-idx-1", - MetadataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey())); + assertNull( + client().admin() + .indices() + .prepareGetSettings("test-idx-1") + .get() + .getSetting("test-idx-1", MetadataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()) + ); - for (ShardStats shardStats: client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { + for (ShardStats shardStats : client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { String historyUUID = shardStats.getCommitStats().getUserData().get(Engine.HISTORY_UUID_KEY); ShardId shardId = shardStats.getShardRouting().shardId(); assertThat(shardStats.getShardRouting() + " doesn't have a history uuid", historyUUID, notNullValue()); @@ -202,8 +212,11 @@ public void testBasicWorkFlow() throws Exception { logger.info("--> delete indices"); cluster().wipeIndices("test-idx-1", "test-idx-2"); logger.info("--> restore one index after deletion"); - restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true) - .setIndices("test-idx-*", "-test-idx-2").execute().actionGet(); + restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .setIndices("test-idx-*", "-test-idx-2") + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); @@ -212,10 +225,15 @@ public void testBasicWorkFlow() throws Exception { assertThat(clusterState.getMetadata().hasIndex("test-idx-1"), equalTo(true)); assertThat(clusterState.getMetadata().hasIndex("test-idx-2"), equalTo(false)); - assertNull(client().admin().indices().prepareGetSettings("test-idx-1").get().getSetting("test-idx-1", - MetadataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey())); + assertNull( + client().admin() + .indices() + .prepareGetSettings("test-idx-1") + .get() + .getSetting("test-idx-1", MetadataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()) + ); - for (ShardStats shardStats: client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { + for (ShardStats shardStats : client().admin().indices().prepareStats(indicesToSnapshot).clear().get().getShards()) { String historyUUID = shardStats.getCommitStats().getUserData().get(Engine.HISTORY_UUID_KEY); ShardId shardId = shardStats.getShardRouting().shardId(); assertThat(shardStats.getShardRouting() + " doesn't have a history uuid", historyUUID, notNullValue()); @@ -245,10 +263,10 @@ public void testSingleGetAfterRestore() throws Exception { createSnapshot(repoName, snapshotName, Collections.singletonList(indexName)); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapshotName) - .setWaitForCompletion(true) - .setRenamePattern(indexName) - .setRenameReplacement(restoredIndexName) - .get(); + .setWaitForCompletion(true) + .setRenamePattern(indexName) + .setRenameReplacement(restoredIndexName) + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertThat(client().prepareGet(restoredIndexName, docId).get().isExists(), equalTo(true)); @@ -258,7 +276,10 @@ public void testFreshIndexUUID() { createRepository("test-repo", "fs"); createIndex("test"); - String originalIndexUUID = client().admin().indices().prepareGetSettings("test").get() + String originalIndexUUID = client().admin() + .indices() + .prepareGetSettings("test") + .get() .getSetting("test", IndexMetadata.SETTING_INDEX_UUID); assertTrue(originalIndexUUID, originalIndexUUID != null); assertFalse(originalIndexUUID, originalIndexUUID.equals(IndexMetadata.INDEX_UUID_NA_VALUE)); @@ -267,10 +288,12 @@ public void testFreshIndexUUID() { NumShards numShards = getNumShards("test"); cluster().wipeIndices("test"); - assertAcked(prepareCreate("test").setSettings(Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries))); + assertAcked(prepareCreate("test").setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries))); ensureGreen(); - String newIndexUUID = client().admin().indices().prepareGetSettings("test").get() + String newIndexUUID = client().admin() + .indices() + .prepareGetSettings("test") + .get() .getSetting("test", IndexMetadata.SETTING_INDEX_UUID); assertTrue(newIndexUUID, newIndexUUID != null); assertFalse(newIndexUUID, newIndexUUID.equals(IndexMetadata.INDEX_UUID_NA_VALUE)); @@ -280,24 +303,40 @@ public void testFreshIndexUUID() { logger.info("--> restore all indices from the snapshot"); RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); ensureGreen(); - String newAfterRestoreIndexUUID = client().admin().indices().prepareGetSettings("test").get() + String newAfterRestoreIndexUUID = client().admin() + .indices() + .prepareGetSettings("test") + .get() .getSetting("test", IndexMetadata.SETTING_INDEX_UUID); - assertTrue("UUID has changed after restore: " + newIndexUUID + " vs. " + newAfterRestoreIndexUUID, - newIndexUUID.equals(newAfterRestoreIndexUUID)); + assertTrue( + "UUID has changed after restore: " + newIndexUUID + " vs. " + newAfterRestoreIndexUUID, + newIndexUUID.equals(newAfterRestoreIndexUUID) + ); logger.info("--> restore indices with different names"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet(); + .setRenamePattern("(.+)") + .setRenameReplacement("$1-copy") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - String copyRestoreUUID = client().admin().indices().prepareGetSettings("test-copy").get() + String copyRestoreUUID = client().admin() + .indices() + .prepareGetSettings("test-copy") + .get() .getSetting("test-copy", IndexMetadata.SETTING_INDEX_UUID); - assertFalse("UUID has been reused on restore: " + copyRestoreUUID + " vs. " + originalIndexUUID, - copyRestoreUUID.equals(originalIndexUUID)); + assertFalse( + "UUID has been reused on restore: " + copyRestoreUUID + " vs. " + originalIndexUUID, + copyRestoreUUID.equals(originalIndexUUID) + ); } public void testEmptySnapshot() throws Exception { @@ -315,20 +354,26 @@ public void testSnapshotFileFailureDuringSnapshot() throws InterruptedException disableRepoConsistencyCheck("This test uses a purposely broken repository so it would fail consistency checks"); logger.info("--> creating repository"); - assertAcked(clusterAdmin().preparePutRepository("test-repo") - .setType("mock").setSettings( - Settings.builder() - .put("location", randomRepoPath()) - .put("random", randomAlphaOfLength(10)) - .put("random_control_io_exception_rate", 0.2)) - .setVerify(false)); + assertAcked( + clusterAdmin().preparePutRepository("test-repo") + .setType("mock") + .setSettings( + Settings.builder() + .put("location", randomRepoPath()) + .put("random", randomAlphaOfLength(10)) + .put("random_control_io_exception_rate", 0.2) + ) + .setVerify(false) + ); createIndexWithRandomDocs("test-idx", 100); logger.info("--> snapshot"); try { CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).setIndices("test-idx").get(); + .setWaitForCompletion(true) + .setIndices("test-idx") + .get(); if (createSnapshotResponse.getSnapshotInfo().totalShards() == createSnapshotResponse.getSnapshotInfo().successfulShards()) { // If we are here, that means we didn't have any failures, let's check it assertThat(getFailureCount("test-repo"), equalTo(0L)); @@ -361,15 +406,22 @@ public void testSnapshotFileFailureDuringSnapshot() throws InterruptedException public void testDataFileFailureDuringSnapshot() throws Exception { disableRepoConsistencyCheck("This test intentionally leaves a broken repository"); - createRepository("test-repo", "mock", - Settings.builder().put("location", randomRepoPath()) - .put("random", randomAlphaOfLength(10)).put("random_data_file_io_exception_rate", 0.3)); + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", randomRepoPath()) + .put("random", randomAlphaOfLength(10)) + .put("random_data_file_io_exception_rate", 0.3) + ); createIndexWithRandomDocs("test-idx", 100); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).setIndices("test-idx").get(); + .setWaitForCompletion(true) + .setIndices("test-idx") + .get(); if (createSnapshotResponse.getSnapshotInfo().totalShards() == createSnapshotResponse.getSnapshotInfo().successfulShards()) { logger.info("--> no failures"); // If we are here, that means we didn't have any failures, let's check it @@ -389,7 +441,8 @@ public void testDataFileFailureDuringSnapshot() throws Exception { // Verify that snapshot status also contains the same failures SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo") - .addSnapshots("test-snap").get(); + .addSnapshots("test-snap") + .get(); assertThat(snapshotsStatusResponse.getSnapshots().size(), equalTo(1)); SnapshotStatus snapshotStatus = snapshotsStatusResponse.getSnapshots().get(0); assertThat(snapshotStatus.getIndices().size(), equalTo(1)); @@ -428,17 +481,24 @@ public void testDataFileFailureDuringRestore() throws Exception { createSnapshot("test-repo", "test-snap", Collections.singletonList("test-idx")); - createRepository("test-repo", "mock", - Settings.builder().put("location", repositoryLocation).put("random", randomAlphaOfLength(10)) - .put("random_data_file_io_exception_rate", 0.3)); + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", repositoryLocation) + .put("random", randomAlphaOfLength(10)) + .put("random_data_file_io_exception_rate", 0.3) + ); // Test restore after index deletion logger.info("--> delete index"); cluster().wipeIndices("test-idx"); logger.info("--> restore index after deletion"); - final RestoreSnapshotResponse restoreResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true) - .get(); + final RestoreSnapshotResponse restoreResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .get(); logger.info("--> total number of simulated failures during restore: [{}]", getFailureCount("test-repo")); final RestoreInfo restoreInfo = restoreResponse.getRestoreInfo(); @@ -456,8 +516,10 @@ public void testDataFileFailureDuringRestore() throws Exception { // block the shard to be assigned again because it failed during restore. final ClusterStateResponse clusterStateResponse = client.admin().cluster().prepareState().get(); assertEquals(1, clusterStateResponse.getState().getNodes().getDataNodes().size()); - assertEquals(restoreInfo.failedShards(), - clusterStateResponse.getState().getRoutingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size()); + assertEquals( + restoreInfo.failedShards(), + clusterStateResponse.getState().getRoutingTable().shardsWithState(ShardRoutingState.UNASSIGNED).size() + ); } } @@ -473,21 +535,32 @@ public void testDataFileCorruptionDuringRestore() throws Exception { createSnapshot("test-repo", "test-snap", Collections.singletonList("test-idx")); - createRepository("test-repo", "mock", - Settings.builder().put("location", repositoryLocation).put("random", randomAlphaOfLength(10)) - .put("use_lucene_corruption", true).put("max_failure_number", 10000000L) - .put("random_data_file_io_exception_rate", 1.0)); + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", repositoryLocation) + .put("random", randomAlphaOfLength(10)) + .put("use_lucene_corruption", true) + .put("max_failure_number", 10000000L) + .put("random_data_file_io_exception_rate", 1.0) + ); // Test restore after index deletion logger.info("--> delete index"); cluster().wipeIndices("test-idx"); logger.info("--> restore corrupt index"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster() + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() .prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().failedShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()) + ); // we have to delete the index here manually, otherwise the cluster will keep // trying to allocate the shards for the index, even though the restore operation // is completed and marked as failed, which can lead to nodes having pending @@ -503,17 +576,19 @@ public void testUnrestorableFilesDuringRestore() throws Exception { final String indexName = "unrestorable-files"; final int maxRetries = randomIntBetween(1, 10); - Settings createIndexSettings = Settings.builder().put(SETTING_ALLOCATION_MAX_RETRY.getKey(), maxRetries) - .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1).build(); + Settings createIndexSettings = Settings.builder() + .put(SETTING_ALLOCATION_MAX_RETRY.getKey(), maxRetries) + .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .build(); Settings repositorySettings = Settings.builder() - .put("random", randomAlphaOfLength(10)) - .put("max_failure_number", 10000000L) - // No lucene corruptions, we want to test retries - .put("use_lucene_corruption", false) - // Restoring a file will never complete - .put("random_data_file_io_exception_rate", 1.0) - .build(); + .put("random", randomAlphaOfLength(10)) + .put("max_failure_number", 10000000L) + // No lucene corruptions, we want to test retries + .put("use_lucene_corruption", false) + // Restoring a file will never complete + .put("random_data_file_io_exception_rate", 1.0) + .build(); Consumer checkUnassignedInfo = unassignedInfo -> { assertThat(unassignedInfo.getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED)); @@ -531,26 +606,36 @@ public void testUnrestorableIndexDuringRestore() throws Exception { final String indexName = "unrestorable-index"; Settings restoreIndexSettings = Settings.builder().put("index.routing.allocation.include._name", randomAlphaOfLength(5)).build(); - Runnable fixupAction =() -> { + Runnable fixupAction = () -> { // remove the shard allocation filtering settings and use the Reroute API to retry the failed shards - assertAcked(client().admin().indices().prepareUpdateSettings(indexName) - .setSettings(Settings.builder() - .putNull("index.routing.allocation.include._name") - .build())); + assertAcked( + client().admin() + .indices() + .prepareUpdateSettings(indexName) + .setSettings(Settings.builder().putNull("index.routing.allocation.include._name").build()) + ); assertAcked(clusterAdmin().prepareReroute().setRetryFailed(true)); }; - unrestorableUseCase(indexName, Settings.EMPTY, Settings.EMPTY, restoreIndexSettings, - unassignedInfo -> assertThat(unassignedInfo.getReason(), equalTo(UnassignedInfo.Reason.NEW_INDEX_RESTORED)), fixupAction); + unrestorableUseCase( + indexName, + Settings.EMPTY, + Settings.EMPTY, + restoreIndexSettings, + unassignedInfo -> assertThat(unassignedInfo.getReason(), equalTo(UnassignedInfo.Reason.NEW_INDEX_RESTORED)), + fixupAction + ); } /** Execute the unrestorable test use case **/ - private void unrestorableUseCase(final String indexName, - final Settings createIndexSettings, - final Settings repositorySettings, - final Settings restoreIndexSettings, - final Consumer checkUnassignedInfo, - final Runnable fixUpAction) throws Exception { + private void unrestorableUseCase( + final String indexName, + final Settings createIndexSettings, + final Settings repositorySettings, + final Settings restoreIndexSettings, + final Consumer checkUnassignedInfo, + final Runnable fixUpAction + ) throws Exception { // create a test repository final Path repositoryLocation = randomRepoPath(); createRepository("test-repo", "fs", repositoryLocation); @@ -570,19 +655,18 @@ private void unrestorableUseCase(final String indexName, assertAcked(client().admin().indices().prepareDelete(indexName)); // update the test repository - assertAcked(clusterAdmin().preparePutRepository("test-repo") - .setType("mock") - .setSettings(Settings.builder() - .put("location", repositoryLocation) - .put(repositorySettings) - .build())); + assertAcked( + clusterAdmin().preparePutRepository("test-repo") + .setType("mock") + .setSettings(Settings.builder().put("location", repositoryLocation).put(repositorySettings).build()) + ); // attempt to restore the snapshot with the given settings RestoreSnapshotResponse restoreResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap") - .setIndices(indexName) - .setIndexSettings(restoreIndexSettings) - .setWaitForCompletion(true) - .get(); + .setIndices(indexName) + .setIndexSettings(restoreIndexSettings) + .setWaitForCompletion(true) + .get(); // check that all shards failed during restore assertThat(restoreResponse.getRestoreInfo().totalShards(), equalTo(numShards.numPrimaries)); @@ -593,8 +677,7 @@ private void unrestorableUseCase(final String indexName, // check that there is no restore in progress RestoreInProgress restoreInProgress = clusterStateResponse.getState().custom(RestoreInProgress.TYPE); assertNotNull("RestoreInProgress must be not null", restoreInProgress); - assertTrue( - "RestoreInProgress must be empty but found entries in " + restoreInProgress, restoreInProgress.isEmpty()); + assertTrue("RestoreInProgress must be empty but found entries in " + restoreInProgress, restoreInProgress.isEmpty()); // check that the shards have been created but are not assigned assertThat(clusterStateResponse.getState().getRoutingTable().allShards(indexName), hasSize(numShards.totalNumShards)); @@ -639,20 +722,27 @@ public void testDeletionOfFailingToRecoverIndexShouldStopRestore() throws Except createSnapshot("test-repo", "test-snap", Collections.singletonList("test-idx")); logger.info("--> update repository with mock version"); - createRepository("test-repo", "mock", Settings.builder() - .put("location", repositoryLocation).put("random", randomAlphaOfLength(10)) - .put("random_data_file_io_exception_rate", 1.0) // Fail completely + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", repositoryLocation) + .put("random", randomAlphaOfLength(10)) + .put("random_data_file_io_exception_rate", 1.0) // Fail completely ); // Test restore after index deletion logger.info("--> delete index"); cluster().wipeIndices("test-idx"); logger.info("--> restore index after deletion"); - ActionFuture restoreSnapshotResponseFuture = - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute(); + ActionFuture restoreSnapshotResponseFuture = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute(); logger.info("--> wait for the index to appear"); - // that would mean that recovery process started and failing + // that would mean that recovery process started and failing waitForIndex("test-idx", TimeValue.timeValueSeconds(10)); logger.info("--> delete index"); @@ -661,15 +751,21 @@ public void testDeletionOfFailingToRecoverIndexShouldStopRestore() throws Except // Now read restore results and make sure it failed RestoreSnapshotResponse restoreSnapshotResponse = restoreSnapshotResponseFuture.actionGet(TimeValue.timeValueSeconds(10)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().failedShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().totalShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().failedShards()) + ); logger.info("--> restoring working repository"); createRepository("test-repo", "fs", repositoryLocation); logger.info("--> trying to restore index again"); - restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); assertDocCount("test-idx", 100L); @@ -680,13 +776,19 @@ public void testUnallocatedShards() { createRepository("test-repo", "fs"); logger.info("--> creating index that cannot be allocated"); - prepareCreate("test-idx", 2, Settings.builder().put(IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + "tag", "nowhere") - .put("index.number_of_shards", 3)).setWaitForActiveShards(ActiveShardCount.NONE).get(); + prepareCreate( + "test-idx", + 2, + Settings.builder() + .put(IndexMetadata.INDEX_ROUTING_INCLUDE_GROUP_SETTING.getKey() + "tag", "nowhere") + .put("index.number_of_shards", 3) + ).setWaitForActiveShards(ActiveShardCount.NONE).get(); logger.info("--> snapshot"); - final SnapshotException sne = expectThrows(SnapshotException.class, - () -> clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).setIndices("test-idx").get()); + final SnapshotException sne = expectThrows( + SnapshotException.class, + () -> clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx").get() + ); assertThat(sne.getMessage(), containsString("Indices don't have primary shards")); assertThat(getRepositoryData("test-repo"), is(RepositoryData.EMPTY)); } @@ -696,9 +798,14 @@ public void testDeleteSnapshot() throws Exception { Client client = client(); Path repo = randomRepoPath(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", repo).put("compress", false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + "test-repo", + "fs", + Settings.builder() + .put("location", repo) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx"); ensureGreen(); @@ -721,11 +828,16 @@ public void testDeleteSnapshot() throws Exception { if (randomBoolean()) { for (int i = 1; i < numberOfSnapshots - 1; i++) { - client.admin().cluster().prepareDeleteSnapshot("test-repo", new String[]{"test-snap-" + i}).get(); + client.admin().cluster().prepareDeleteSnapshot("test-repo", new String[] { "test-snap-" + i }).get(); } } else { - client.admin().cluster().prepareDeleteSnapshot( - "test-repo", IntStream.range(1, numberOfSnapshots - 1).mapToObj(i -> "test-snap-" + i).toArray(String[]::new)).get(); + client.admin() + .cluster() + .prepareDeleteSnapshot( + "test-repo", + IntStream.range(1, numberOfSnapshots - 1).mapToObj(i -> "test-snap-" + i).toArray(String[]::new) + ) + .get(); } int numberOfFilesAfterDeletion = numberOfFiles(repo); @@ -737,8 +849,12 @@ public void testDeleteSnapshot() throws Exception { logger.info("--> restore index"); String lastSnapshot = "test-snap-" + (numberOfSnapshots - 1); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", lastSnapshot) - .setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", lastSnapshot) + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx", 10L * numberOfSnapshots); @@ -768,8 +884,11 @@ public void testSnapshotClosedIndex() throws Exception { public void testMoveShardWhileSnapshotting() throws Exception { Client client = client(); Path repositoryLocation = randomRepoPath(); - createRepository("test-repo", "mock", Settings.builder().put("location", repositoryLocation) - .put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200)); + createRepository( + "test-repo", + "mock", + Settings.builder().put("location", repositoryLocation).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); // Create index on 2 nodes and make sure each node has a primary by setting no replicas assertAcked(prepareCreate("test-idx", 2, Settings.builder().put("number_of_replicas", 0))); @@ -802,8 +921,12 @@ public void testMoveShardWhileSnapshotting() throws Exception { createRepository("test-repo", "fs", repositoryLocation); logger.info("--> restore index"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx", 100L); } @@ -812,9 +935,11 @@ public void testDeleteRepositoryWhileSnapshotting() throws Exception { disableRepoConsistencyCheck("This test uses a purposely broken repository so it would fail consistency checks"); Client client = client(); Path repositoryLocation = randomRepoPath(); - createRepository("test-repo", "mock", - Settings.builder().put("location", repositoryLocation).put("random", randomAlphaOfLength(10)) - .put("wait_after_unblock", 200)); + createRepository( + "test-repo", + "mock", + Settings.builder().put("location", repositoryLocation).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); // Create index on 2 nodes and make sure each node has a primary by setting no replicas assertAcked(prepareCreate("test-idx", 2, Settings.builder().put("number_of_replicas", 0))); @@ -837,24 +962,34 @@ public void testDeleteRepositoryWhileSnapshotting() throws Exception { fail("shouldn't be able to delete in-use repository"); } catch (Exception ex) { logger.info("--> in-use repository deletion failed"); - assertThat(ex.getMessage(), - equalTo("trying to modify or unregister repository [test-repo] that is currently used (snapshot is in progress)")); + assertThat( + ex.getMessage(), + equalTo("trying to modify or unregister repository [test-repo] that is currently used (snapshot is in progress)") + ); } logger.info("--> trying to move repository to another location"); try { - client.admin().cluster().preparePutRepository("test-repo") - .setType("fs").setSettings(Settings.builder().put("location", repositoryLocation.resolve("test")) - ).get(); + client.admin() + .cluster() + .preparePutRepository("test-repo") + .setType("fs") + .setSettings(Settings.builder().put("location", repositoryLocation.resolve("test"))) + .get(); fail("shouldn't be able to replace in-use repository"); } catch (Exception ex) { logger.info("--> in-use repository replacement failed"); } logger.info("--> trying to create a repository with different name"); - assertAcked(client.admin().cluster().preparePutRepository("test-repo-2") + assertAcked( + client.admin() + .cluster() + .preparePutRepository("test-repo-2") .setVerify(false) // do not do verification itself as snapshot threads could be fully blocked - .setType("fs").setSettings(Settings.builder().put("location", repositoryLocation.resolve("test")))); + .setType("fs") + .setSettings(Settings.builder().put("location", repositoryLocation.resolve("test"))) + ); logger.info("--> unblocking blocked node"); unblockNode("test-repo", blockedNode); @@ -871,8 +1006,12 @@ public void testDeleteRepositoryWhileSnapshotting() throws Exception { createRepository("test-repo", "fs", repositoryLocation); logger.info("--> restore index"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx", 100); } @@ -891,14 +1030,23 @@ public void testReadonlyRepository() throws Exception { logger.info("--> delete index"); cluster().wipeIndices("test-idx"); - createRepository("readonly-repo", "fs", Settings.builder() - .put("location", repositoryLocation) - .put("compress", randomBoolean()) - .put(READONLY_SETTING_KEY, true) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + "readonly-repo", + "fs", + Settings.builder() + .put("location", repositoryLocation) + .put("compress", randomBoolean()) + .put(READONLY_SETTING_KEY, true) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); logger.info("--> restore index after deletion"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("readonly-repo", "test-snap") - .setWaitForCompletion(true).setIndices("test-idx").execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("readonly-repo", "test-snap") + .setWaitForCompletion(true) + .setIndices("test-idx") + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx", 100L); @@ -908,14 +1056,22 @@ public void testReadonlyRepository() throws Exception { assertThat(getSnapshotsResponse.getSnapshots("readonly-repo").size(), equalTo(1)); logger.info("--> try deleting snapshot"); - assertRequestBuilderThrows(client.admin().cluster().prepareDeleteSnapshot("readonly-repo", "test-snap"), RepositoryException.class, - "cannot delete snapshot from a readonly repository"); + assertRequestBuilderThrows( + client.admin().cluster().prepareDeleteSnapshot("readonly-repo", "test-snap"), + RepositoryException.class, + "cannot delete snapshot from a readonly repository" + ); logger.info("--> try making another snapshot"); - assertRequestBuilderThrows(client.admin().cluster().prepareCreateSnapshot("readonly-repo", "test-snap-2") - .setWaitForCompletion(true).setIndices("test-idx"), + assertRequestBuilderThrows( + client.admin() + .cluster() + .prepareCreateSnapshot("readonly-repo", "test-snap-2") + .setWaitForCompletion(true) + .setIndices("test-idx"), RepositoryException.class, - "cannot create snapshot in a readonly repository"); + "cannot create snapshot in a readonly repository" + ); } public void testThrottling() throws Exception { @@ -924,13 +1080,16 @@ public void testThrottling() throws Exception { boolean throttleSnapshot = randomBoolean(); boolean throttleRestore = randomBoolean(); boolean throttleRestoreViaRecoverySettings = throttleRestore && randomBoolean(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", randomRepoPath()) - .put("compress", randomBoolean()) - .put("chunk_size", randomIntBetween(1000, 10000), ByteSizeUnit.BYTES) - .put("max_restore_bytes_per_sec", - throttleRestore && (throttleRestoreViaRecoverySettings == false) ? "10k" : "0") - .put("max_snapshot_bytes_per_sec", throttleSnapshot ? "10k" : "0")); + createRepository( + "test-repo", + "fs", + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(1000, 10000), ByteSizeUnit.BYTES) + .put("max_restore_bytes_per_sec", throttleRestore && (throttleRestoreViaRecoverySettings == false) ? "10k" : "0") + .put("max_snapshot_bytes_per_sec", throttleSnapshot ? "10k" : "0") + ); createIndexWithRandomDocs("test-idx", 100); createSnapshot("test-repo", "test-snap", Collections.singletonList("test-idx")); @@ -939,11 +1098,21 @@ public void testThrottling() throws Exception { cluster().wipeIndices("test-idx"); logger.info("--> restore index"); - client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), - throttleRestoreViaRecoverySettings ? "10k" : "0").build()).get(); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().actionGet(); + client.admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder() + .put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), throttleRestoreViaRecoverySettings ? "10k" : "0") + .build() + ) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertDocCount("test-idx", 100L); @@ -965,15 +1134,20 @@ public void testThrottling() throws Exception { } else { assertThat(restorePause, equalTo(0L)); } - client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .putNull(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()).build()).get(); + client.admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()).build()) + .get(); } public void testSnapshotStatus() throws Exception { Client client = client(); - createRepository("test-repo", "mock", - Settings.builder().put("location", randomRepoPath()) - .put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200)); + createRepository( + "test-repo", + "mock", + Settings.builder().put("location", randomRepoPath()).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); // Create index on 2 nodes and make sure each node has a primary by setting no replicas assertAcked(prepareCreate("test-idx", 2, Settings.builder().put("number_of_replicas", 0))); @@ -983,14 +1157,18 @@ public void testSnapshotStatus() throws Exception { String blockedNode = blockNodeWithIndex("test-repo", "test-idx"); logger.info("--> snapshot"); - client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(false).setIncludeGlobalState(false).setIndices("test-idx").get(); + client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(false) + .setIncludeGlobalState(false) + .setIndices("test-idx") + .get(); logger.info("--> waiting for block to kick in"); waitForBlock(blockedNode, "test-repo"); - logger.info("--> execution was blocked on node [{}], checking snapshot status with specified repository and snapshot", - blockedNode); + logger.info("--> execution was blocked on node [{}], checking snapshot status with specified repository and snapshot", blockedNode); SnapshotsStatusResponse response = client.admin().cluster().prepareSnapshotStatus("test-repo").execute().actionGet(); assertThat(response.getSnapshots().size(), equalTo(1)); SnapshotStatus snapshotStatus = response.getSnapshots().get(0); @@ -1021,8 +1199,12 @@ public void testSnapshotStatus() throws Exception { } logger.info("--> checking that _current returns the currently running snapshot"); - GetSnapshotsResponse getResponse = client.admin().cluster().prepareGetSnapshots("test-repo") - .setCurrentSnapshot().execute().actionGet(); + GetSnapshotsResponse getResponse = client.admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setCurrentSnapshot() + .execute() + .actionGet(); assertThat(getResponse.getSnapshots("test-repo").size(), equalTo(1)); SnapshotInfo snapshotInfo = getResponse.getSnapshots("test-repo").get(0); assertThat(snapshotInfo.state(), equalTo(SnapshotState.IN_PROGRESS)); @@ -1052,26 +1234,40 @@ public void testSnapshotStatus() throws Exception { assertThat(response.getSnapshots().size(), equalTo(0)); logger.info("--> checking that _current no longer returns the snapshot"); - assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("_current") - .execute().actionGet().getSnapshots("test-repo").isEmpty(), - equalTo(true)); + assertThat( + client.admin() + .cluster() + .prepareGetSnapshots("test-repo") + .addSnapshots("_current") + .execute() + .actionGet() + .getSnapshots("test-repo") + .isEmpty(), + equalTo(true) + ); // test that getting an unavailable snapshot status throws an exception if ignoreUnavailable is false on the request - SnapshotMissingException ex = expectThrows(SnapshotMissingException.class, () -> - client.admin().cluster().prepareSnapshotStatus("test-repo").addSnapshots("test-snap-doesnt-exist").get()); + SnapshotMissingException ex = expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareSnapshotStatus("test-repo").addSnapshots("test-snap-doesnt-exist").get() + ); assertEquals("[test-repo:test-snap-doesnt-exist] is missing", ex.getMessage()); // test that getting an unavailable snapshot status does not throw an exception if ignoreUnavailable is true on the request - response = client.admin().cluster().prepareSnapshotStatus("test-repo") - .addSnapshots("test-snap-doesnt-exist") - .setIgnoreUnavailable(true) - .get(); + response = client.admin() + .cluster() + .prepareSnapshotStatus("test-repo") + .addSnapshots("test-snap-doesnt-exist") + .setIgnoreUnavailable(true) + .get(); assertTrue(response.getSnapshots().isEmpty()); // test getting snapshot status for available and unavailable snapshots where ignoreUnavailable is true // (available one should be returned) - response = client.admin().cluster().prepareSnapshotStatus("test-repo") - .addSnapshots("test-snap", "test-snap-doesnt-exist") - .setIgnoreUnavailable(true) - .get(); + response = client.admin() + .cluster() + .prepareSnapshotStatus("test-repo") + .addSnapshots("test-snap", "test-snap-doesnt-exist") + .setIgnoreUnavailable(true) + .get(); assertEquals(1, response.getSnapshots().size()); assertEquals("test-snap", response.getSnapshots().get(0).getSnapshot().getSnapshotId().getName()); } @@ -1091,9 +1287,11 @@ public void testSnapshotRelocatingPrimary() throws Exception { logger.info("--> wait for relocations to start"); - assertBusy(() -> assertThat( - clusterAdmin().prepareHealth("test-idx").execute().actionGet().getRelocatingShards(), greaterThan(0)), - 1L, TimeUnit.MINUTES); + assertBusy( + () -> assertThat(clusterAdmin().prepareHealth("test-idx").execute().actionGet().getRelocatingShards(), greaterThan(0)), + 1L, + TimeUnit.MINUTES + ); logger.info("--> snapshot"); client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(false).setIndices("test-idx").get(); @@ -1111,8 +1309,7 @@ public void testSnapshotMoreThanOnce() throws InterruptedException { createRepository("test-repo", "fs"); // only one shard - final Settings indexSettings = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build(); + final Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build(); assertAcked(prepareCreate("test").setSettings(indexSettings)); ensureGreen(); @@ -1122,8 +1319,13 @@ public void testSnapshotMoreThanOnce() throws InterruptedException { createSnapshot("test-repo", "test", Collections.singletonList("test")); assertThat(getSnapshot("test-repo", "test").state(), equalTo(SnapshotState.SUCCESS)); { - SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo") - .setSnapshots("test").get().getSnapshots().get(0); + SnapshotStatus snapshotStatus = client.admin() + .cluster() + .prepareSnapshotStatus("test-repo") + .setSnapshots("test") + .get() + .getSnapshots() + .get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { assertThat(status.getStats().getProcessedFileCount(), greaterThan(1)); @@ -1133,8 +1335,13 @@ public void testSnapshotMoreThanOnce() throws InterruptedException { createSnapshot("test-repo", "test-1", Collections.singletonList("test")); assertThat(getSnapshot("test-repo", "test-1").state(), equalTo(SnapshotState.SUCCESS)); { - SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo") - .setSnapshots("test-1").get().getSnapshots().get(0); + SnapshotStatus snapshotStatus = client.admin() + .cluster() + .prepareSnapshotStatus("test-repo") + .setSnapshots("test-1") + .get() + .getSnapshots() + .get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { assertThat(status.getStats().getProcessedFileCount(), equalTo(0)); @@ -1145,8 +1352,13 @@ public void testSnapshotMoreThanOnce() throws InterruptedException { createSnapshot("test-repo", "test-2", Collections.singletonList("test")); assertThat(getSnapshot("test-repo", "test-2").state(), equalTo(SnapshotState.SUCCESS)); { - SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo") - .setSnapshots("test-2").get().getSnapshots().get(0); + SnapshotStatus snapshotStatus = client.admin() + .cluster() + .prepareSnapshotStatus("test-repo") + .setSnapshots("test-2") + .get() + .getSnapshots() + .get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { // we flush before the snapshot such that we have to process the segments_N files plus the .del file @@ -1159,17 +1371,25 @@ public void testSnapshotMoreThanOnce() throws InterruptedException { public void testCloseOrDeleteIndexDuringSnapshot() throws Exception { disableRepoConsistencyCheck("This test intentionally leaves a broken repository"); - createRepository("test-repo", "mock", Settings.builder() - .put("location", randomRepoPath()).put("compress", randomBoolean()) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) - .put("block_on_data", true)); + createRepository( + "test-repo", + "mock", + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + .put("block_on_data", true) + ); createIndexWithRandomDocs("test-idx-1", 100); createIndexWithRandomDocs("test-idx-2", 100); createIndexWithRandomDocs("test-idx-3", 100); logger.info("--> snapshot"); ActionFuture future = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap") - .setIndices("test-idx-*").setWaitForCompletion(true).setPartial(false).execute(); + .setIndices("test-idx-*") + .setWaitForCompletion(true) + .setPartial(false) + .execute(); logger.info("--> wait for block to kick in"); waitForBlockOnAnyDataNode("test-repo"); @@ -1222,9 +1442,7 @@ public void testCloseIndexDuringRestore() throws Exception { final ActionFuture restoreFut; try { logger.info("--> start restore"); - restoreFut = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true) - .execute(); + restoreFut = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute(); logger.info("--> waiting for block to kick in"); waitForBlockOnAnyDataNode("test-repo"); @@ -1263,11 +1481,29 @@ public void testDeleteSnapshotWhileRestoringFails() throws Exception { logger.info("--> take snapshots"); final String snapshotName = "test-snap"; - assertThat(client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName) - .setIndices(indexName).setWaitForCompletion(true).get().getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + assertThat( + client.admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotName) + .setIndices(indexName) + .setWaitForCompletion(true) + .get() + .getSnapshotInfo() + .state(), + equalTo(SnapshotState.SUCCESS) + ); final String snapshotName2 = "test-snap-2"; - assertThat(client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName2) - .setIndices(indexName).setWaitForCompletion(true).get().getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + assertThat( + client.admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotName2) + .setIndices(indexName) + .setWaitForCompletion(true) + .get() + .getSnapshotInfo() + .state(), + equalTo(SnapshotState.SUCCESS) + ); logger.info("--> delete index before restoring"); assertAcked(client.admin().indices().prepareDelete(indexName).get()); @@ -1278,16 +1514,16 @@ public void testDeleteSnapshotWhileRestoringFails() throws Exception { final ActionFuture restoreFut; try { logger.info("--> start restore"); - restoreFut = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName) - .setWaitForCompletion(true) - .execute(); + restoreFut = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName).setWaitForCompletion(true).execute(); logger.info("--> waiting for block to kick in"); waitForBlockOnAnyDataNode(repoName); logger.info("--> try deleting the snapshot while the restore is in progress (should throw an error)"); - ConcurrentSnapshotExecutionException e = expectThrows(ConcurrentSnapshotExecutionException.class, () -> - clusterAdmin().prepareDeleteSnapshot(repoName, snapshotName).get()); + ConcurrentSnapshotExecutionException e = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> clusterAdmin().prepareDeleteSnapshot(repoName, snapshotName).get() + ); assertEquals(repoName, e.getRepositoryName()); assertEquals(snapshotName, e.getSnapshotName()); assertThat(e.getMessage(), containsString("cannot delete snapshot during a restore")); @@ -1305,7 +1541,8 @@ private void waitForIndex(final String index, TimeValue timeout) throws Exceptio assertBusy( () -> assertTrue("Expected index [" + index + "] to exist", indexExists(index)), timeout.millis(), - TimeUnit.MILLISECONDS); + TimeUnit.MILLISECONDS + ); } public void testSnapshotName() throws Exception { @@ -1315,15 +1552,16 @@ public void testSnapshotName() throws Exception { createRepository("test-repo", "fs"); - expectThrows(InvalidSnapshotNameException.class, - () -> client.admin().cluster().prepareCreateSnapshot("test-repo", "_foo").get()); - expectThrows(SnapshotMissingException.class, - () -> client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("_foo") - .get().getSnapshots("test-repo")); - expectThrows(SnapshotMissingException.class, - () -> client.admin().cluster().prepareDeleteSnapshot("test-repo", "_foo").get()); - expectThrows(SnapshotMissingException.class, - () -> client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("_foo").get()); + expectThrows(InvalidSnapshotNameException.class, () -> client.admin().cluster().prepareCreateSnapshot("test-repo", "_foo").get()); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("_foo").get().getSnapshots("test-repo") + ); + expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareDeleteSnapshot("test-repo", "_foo").get()); + expectThrows( + SnapshotMissingException.class, + () -> client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("_foo").get() + ); } public void testListCorruptedSnapshot() throws Exception { @@ -1331,36 +1569,46 @@ public void testListCorruptedSnapshot() throws Exception { Client client = client(); Path repo = randomRepoPath(); - createRepository("test-repo", "fs", Settings.builder() - .put("location", repo).put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)); + createRepository( + "test-repo", + "fs", + Settings.builder().put("location", repo).put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + ); createIndex("test-idx-1", "test-idx-2", "test-idx-3"); logger.info("--> indexing some data"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar"), - client().prepareIndex("test-idx-3").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar"), + client().prepareIndex("test-idx-3").setSource("foo", "bar") + ); createSnapshot("test-repo", "test-snap-1", Collections.singletonList("test-idx-*")); final SnapshotInfo snapshotInfo = createSnapshot("test-repo", "test-snap-2", Collections.singletonList("test-idx-*")); logger.info("--> truncate snapshot file to make it unreadable"); Path snapshotPath = repo.resolve("snap-" + snapshotInfo.snapshotId().getUUID() + ".dat"); - try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) { + try (SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) { outChan.truncate(randomInt(10)); } logger.info("--> get snapshots request should return both snapshots"); - List snapshotInfos = client.admin().cluster() - .prepareGetSnapshots("test-repo") - .setIgnoreUnavailable(true).get().getSnapshots("test-repo"); + List snapshotInfos = client.admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setIgnoreUnavailable(true) + .get() + .getSnapshots("test-repo"); assertThat(snapshotInfos.size(), equalTo(1)); assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo("test-snap-1")); - final SnapshotException ex = expectThrows(SnapshotException.class, () -> - client.admin().cluster().prepareGetSnapshots("test-repo").setIgnoreUnavailable(false).get().getSnapshots("test-repo")); + final SnapshotException ex = expectThrows( + SnapshotException.class, + () -> client.admin().cluster().prepareGetSnapshots("test-repo").setIgnoreUnavailable(false).get().getSnapshots("test-repo") + ); assertThat(ex.getRepositoryName(), equalTo("test-repo")); assertThat(ex.getSnapshotName(), equalTo("test-snap-2")); } @@ -1372,17 +1620,19 @@ public void testRestoreSnapshotWithCorruptedGlobalState() throws Exception { createRepository(repoName, "fs", repo); createIndex("test-idx-1", "test-idx-2"); - indexRandom(true, - client().prepareIndex("test-idx-1").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar"), - client().prepareIndex("test-idx-2").setSource("foo", "bar")); + indexRandom( + true, + client().prepareIndex("test-idx-1").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar"), + client().prepareIndex("test-idx-2").setSource("foo", "bar") + ); flushAndRefresh("test-idx-1", "test-idx-2"); final String snapshotName = "test-snap"; final SnapshotInfo snapshotInfo = createFullSnapshot(repoName, snapshotName); final Path globalStatePath = repo.resolve("meta-" + snapshotInfo.snapshotId().getUUID() + ".dat"); - try(SeekableByteChannel outChan = Files.newByteChannel(globalStatePath, StandardOpenOption.WRITE)) { + try (SeekableByteChannel outChan = Files.newByteChannel(globalStatePath, StandardOpenOption.WRITE)) { outChan.truncate(randomInt(10)); } @@ -1391,18 +1641,16 @@ public void testRestoreSnapshotWithCorruptedGlobalState() throws Exception { assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo(snapshotName)); - SnapshotsStatusResponse snapshotStatusResponse = - clusterAdmin().prepareSnapshotStatus(repoName).setSnapshots(snapshotName).get(); + SnapshotsStatusResponse snapshotStatusResponse = clusterAdmin().prepareSnapshotStatus(repoName).setSnapshots(snapshotName).get(); assertThat(snapshotStatusResponse.getSnapshots(), hasSize(1)); assertThat(snapshotStatusResponse.getSnapshots().get(0).getSnapshot().getSnapshotId().getName(), equalTo(snapshotName)); assertAcked(client().admin().indices().prepareDelete("test-idx-1", "test-idx-2")); - SnapshotException ex = expectThrows(SnapshotException.class, () -> clusterAdmin() - .prepareRestoreSnapshot(repoName, snapshotName) - .setRestoreGlobalState(true) - .setWaitForCompletion(true) - .get()); + SnapshotException ex = expectThrows( + SnapshotException.class, + () -> clusterAdmin().prepareRestoreSnapshot(repoName, snapshotName).setRestoreGlobalState(true).setWaitForCompletion(true).get() + ); assertThat(ex.getRepositoryName(), equalTo(repoName)); assertThat(ex.getSnapshotName(), equalTo(snapshotName)); assertThat(ex.getMessage(), containsString("failed to read global metadata")); @@ -1457,10 +1705,11 @@ public void testRestoreSnapshotWithCorruptedIndexMetadata() throws Exception { final Path indexMetadataPath = repo.resolve("indices") .resolve(corruptedIndex.getId()) .resolve( - "meta-" + repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotInfo.snapshotId(), corruptedIndex) + ".dat"); + "meta-" + repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotInfo.snapshotId(), corruptedIndex) + ".dat" + ); // Truncate the index metadata file - try(SeekableByteChannel outChan = Files.newByteChannel(indexMetadataPath, StandardOpenOption.WRITE)) { + try (SeekableByteChannel outChan = Files.newByteChannel(indexMetadataPath, StandardOpenOption.WRITE)) { outChan.truncate(randomInt(10)); } @@ -1509,13 +1758,15 @@ public void testCannotCreateSnapshotsWithSameName() throws Exception { logger.info("--> second snapshot of the same name should fail"); try { CreateSnapshotResponse createSnapshotResponse = client.admin() - .cluster() - .prepareCreateSnapshot(repositoryName, snapshotName) - .setWaitForCompletion(true) - .setIndices(indexName) - .get(); - fail("should not be allowed to create a snapshot with the same name as an already existing snapshot: " + - createSnapshotResponse.getSnapshotInfo().snapshotId()); + .cluster() + .prepareCreateSnapshot(repositoryName, snapshotName) + .setWaitForCompletion(true) + .setIndices(indexName) + .get(); + fail( + "should not be allowed to create a snapshot with the same name as an already existing snapshot: " + + createSnapshotResponse.getSnapshotInfo().snapshotId() + ); } catch (InvalidSnapshotNameException e) { assertThat(e.getMessage(), containsString("snapshot with the same name already exists")); } @@ -1541,13 +1792,17 @@ public void testSnapshotCanceledOnRemovedShard() throws Exception { final String index = "test-idx"; final String snapshot = "test-snap"; - assertAcked(prepareCreate(index, 1, - Settings.builder().put("number_of_shards", numPrimaries).put("number_of_replicas", numReplicas))); + assertAcked( + prepareCreate(index, 1, Settings.builder().put("number_of_shards", numPrimaries).put("number_of_replicas", numReplicas)) + ); indexRandomDocs(index, 100); - createRepository(repo, "mock", Settings.builder() - .put("location", randomRepoPath()).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200)); + createRepository( + repo, + "mock", + Settings.builder().put("location", randomRepoPath()).put("random", randomAlphaOfLength(10)).put("wait_after_unblock", 200) + ); String blockedNode = blockNodeWithIndex(repo, index); @@ -1581,33 +1836,46 @@ public void testSnapshotCanceledOnRemovedShard() throws Exception { public void testSnapshotSucceedsAfterSnapshotFailure() throws Exception { // TODO: Fix repo cleanup logic to handle these leaked snap-file and only exclude test-repo (the mock repo) here. disableRepoConsistencyCheck( - "This test uses a purposely broken repository implementation that results in leaking snap-{uuid}.dat files"); + "This test uses a purposely broken repository implementation that results in leaking snap-{uuid}.dat files" + ); logger.info("--> creating repository"); final Path repoPath = randomRepoPath(); final Client client = client(); - assertAcked(client.admin().cluster().preparePutRepository("test-repo").setType("mock").setVerify(false).setSettings( - Settings.builder() - .put("location", repoPath) - .put("random_control_io_exception_rate", randomIntBetween(5, 20) / 100f) - // test that we can take a snapshot after a failed one, even if a partial index-N was written - .put("random", randomAlphaOfLength(10)))); - - assertAcked(prepareCreate("test-idx").setSettings( - // the less the number of shards, the less control files we have, so we are giving a higher probability of - // triggering an IOException toward the end when writing the pending-index-* files, which are the files - // that caused problems with writing subsequent snapshots if they happened to be lingering in the repository - indexSettingsNoReplicas(1))); + assertAcked( + client.admin() + .cluster() + .preparePutRepository("test-repo") + .setType("mock") + .setVerify(false) + .setSettings( + Settings.builder() + .put("location", repoPath) + .put("random_control_io_exception_rate", randomIntBetween(5, 20) / 100f) + // test that we can take a snapshot after a failed one, even if a partial index-N was written + .put("random", randomAlphaOfLength(10)) + ) + ); + + assertAcked( + prepareCreate("test-idx").setSettings( + // the less the number of shards, the less control files we have, so we are giving a higher probability of + // triggering an IOException toward the end when writing the pending-index-* files, which are the files + // that caused problems with writing subsequent snapshots if they happened to be lingering in the repository + indexSettingsNoReplicas(1) + ) + ); ensureGreen(); final int numDocs = randomIntBetween(1, 5); indexRandomDocs("test-idx", numDocs); logger.info("--> snapshot with potential I/O failures"); try { - CreateSnapshotResponse createSnapshotResponse = - client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true) - .setIndices("test-idx") - .get(); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .setIndices("test-idx") + .get(); if (createSnapshotResponse.getSnapshotInfo().totalShards() != createSnapshotResponse.getSnapshotInfo().successfulShards()) { assertThat(getFailureCount("test-repo"), greaterThan(0L)); assertThat(createSnapshotResponse.getSnapshotInfo().shardFailures().size(), greaterThan(0)); @@ -1629,11 +1897,14 @@ public void testGetSnapshotsFromIndexBlobOnly() throws Exception { logger.info("--> creating repository"); final Path repoPath = randomRepoPath(); final Client client = client(); - assertAcked(client.admin().cluster() - .preparePutRepository("test-repo") - .setType("fs") - .setVerify(false) - .setSettings(Settings.builder().put("location", repoPath))); + assertAcked( + client.admin() + .cluster() + .preparePutRepository("test-repo") + .setType("fs") + .setVerify(false) + .setSettings(Settings.builder().put("location", repoPath)) + ); logger.info("--> creating random number of indices"); final int numIndices = randomIntBetween(1, 10); @@ -1670,30 +1941,18 @@ public void testGetSnapshotsFromIndexBlobOnly() throws Exception { } logger.info("--> verify _all returns snapshot info"); - GetSnapshotsResponse response = clusterAdmin() - .prepareGetSnapshots("test-repo") - .setSnapshots("_all") - .setVerbose(false) - .get(); + GetSnapshotsResponse response = clusterAdmin().prepareGetSnapshots("test-repo").setSnapshots("_all").setVerbose(false).get(); assertEquals(indicesPerSnapshot.size(), response.getSnapshots("test-repo").size()); verifySnapshotInfo(response, indicesPerSnapshot); logger.info("--> verify wildcard returns snapshot info"); - response = clusterAdmin() - .prepareGetSnapshots("test-repo") - .setSnapshots("test-snap-*") - .setVerbose(false) - .get(); + response = clusterAdmin().prepareGetSnapshots("test-repo").setSnapshots("test-snap-*").setVerbose(false).get(); assertEquals(indicesPerSnapshot.size(), response.getSnapshots("test-repo").size()); verifySnapshotInfo(response, indicesPerSnapshot); logger.info("--> verify individual requests return snapshot info"); for (int i = 0; i < numSnapshots; i++) { - response = clusterAdmin() - .prepareGetSnapshots("test-repo") - .setSnapshots("test-snap-" + i) - .setVerbose(false) - .get(); + response = clusterAdmin().prepareGetSnapshots("test-repo").setSnapshots("test-snap-" + i).setVerbose(false).get(); assertEquals(1, response.getSnapshots("test-repo").size()); verifySnapshotInfo(response, indicesPerSnapshot); } @@ -1732,8 +1991,12 @@ public void testSnapshottingWithMissingSequenceNumbers() throws Exception { assertAcked(client.admin().indices().prepareDelete(indexName)); logger.info("--> restore all indices from the snapshot"); - RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap") - .setWaitForCompletion(true).execute().get(); + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setWaitForCompletion(true) + .execute() + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); IndicesStatsResponse stats = client().admin().indices().prepareStats(indexName).clear().get(); @@ -1789,12 +2052,20 @@ public void testSnapshotDifferentIndicesBySameName() throws InterruptedException assertThat(snapshot2.successfulShards(), is(newShardCount)); logger.info("--> restoring snapshot 1"); - clusterAdmin().prepareRestoreSnapshot(repoName, "snap-1").setIndices(indexName).setRenamePattern(indexName) - .setRenameReplacement("restored-1").setWaitForCompletion(true).get(); + clusterAdmin().prepareRestoreSnapshot(repoName, "snap-1") + .setIndices(indexName) + .setRenamePattern(indexName) + .setRenameReplacement("restored-1") + .setWaitForCompletion(true) + .get(); logger.info("--> restoring snapshot 2"); - clusterAdmin().prepareRestoreSnapshot(repoName, "snap-2").setIndices(indexName).setRenamePattern(indexName) - .setRenameReplacement("restored-2").setWaitForCompletion(true).get(); + clusterAdmin().prepareRestoreSnapshot(repoName, "snap-2") + .setIndices(indexName) + .setRenamePattern(indexName) + .setRenameReplacement("restored-2") + .setWaitForCompletion(true) + .get(); logger.info("--> verify doc counts"); assertDocCount("restored-1", docCount); @@ -1814,8 +2085,12 @@ public void testSnapshotDifferentIndicesBySameName() throws InterruptedException } assertAcked(startDeleteSnapshot(repoName, snapshotToDelete).get()); logger.info("--> restoring snapshot [{}]", snapshotToRestore); - clusterAdmin().prepareRestoreSnapshot(repoName, snapshotToRestore).setIndices(indexName).setRenamePattern(indexName) - .setRenameReplacement("restored-3").setWaitForCompletion(true).get(); + clusterAdmin().prepareRestoreSnapshot(repoName, snapshotToRestore) + .setIndices(indexName) + .setRenamePattern(indexName) + .setRenameReplacement("restored-3") + .setWaitForCompletion(true) + .get(); logger.info("--> verify doc counts"); assertDocCount("restored-3", expectedCount); @@ -1825,7 +2100,7 @@ public void testBulkDeleteWithOverlappingPatterns() { final int numberOfSnapshots = between(5, 15); createRepository("test-repo", "fs"); - final String[] indices = {"test-idx-1", "test-idx-2", "test-idx-3"}; + final String[] indices = { "test-idx-1", "test-idx-2", "test-idx-3" }; createIndex(indices); ensureGreen(); @@ -1854,10 +2129,12 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { createRepository(repoName, "fs"); logger.info("--> creating indices"); - createIndex(normalIndex, indexSettingsNoReplicas(randomIntBetween(1,3)).build()); - createIndex(hiddenIndex, indexSettingsNoReplicas(randomIntBetween(1,3)).put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build()); - createIndex(dottedHiddenIndex, - indexSettingsNoReplicas(randomIntBetween(1,3)).put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build()); + createIndex(normalIndex, indexSettingsNoReplicas(randomIntBetween(1, 3)).build()); + createIndex(hiddenIndex, indexSettingsNoReplicas(randomIntBetween(1, 3)).put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build()); + createIndex( + dottedHiddenIndex, + indexSettingsNoReplicas(randomIntBetween(1, 3)).put(IndexMetadata.SETTING_INDEX_HIDDEN, true).build() + ); ensureGreen(); indexRandomDocs(normalIndex, 100); @@ -1868,8 +2145,12 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { final String snapName = "test-snap"; createSnapshot(repoName, snapName, Collections.singletonList(randomFrom("*", "_all"))); - List snapshotInfos = client.admin().cluster().prepareGetSnapshots(repoName) - .setSnapshots(randomFrom(snapName, "_all", "*", "*-snap", "test*")).get().getSnapshots(repoName); + List snapshotInfos = client.admin() + .cluster() + .prepareGetSnapshots(repoName) + .setSnapshots(randomFrom(snapName, "_all", "*", "*-snap", "test*")) + .get() + .getSnapshots(repoName); assertThat(snapshotInfos.size(), equalTo(1)); SnapshotInfo snapshotInfo = snapshotInfos.get(0); assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); @@ -1880,14 +2161,16 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { // Verify that hidden indices get restored with a wildcard restore { - RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin() - .prepareRestoreSnapshot(repoName, snapName) + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapName) .setWaitForCompletion(true) .setIndices("*") - .execute().get(); + .execute() + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()) + ); assertThat(restoreSnapshotResponse.getRestoreInfo().indices(), containsInAnyOrder(normalIndex, hiddenIndex, dottedHiddenIndex)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetadata().hasIndex(normalIndex), equalTo(true)); @@ -1898,14 +2181,16 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { // Verify that exclusions work on hidden indices { - RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin() - .prepareRestoreSnapshot(repoName, snapName) + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapName) .setWaitForCompletion(true) .setIndices("*", "-.*") - .execute().get(); + .execute() + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()) + ); assertThat(restoreSnapshotResponse.getRestoreInfo().indices(), containsInAnyOrder(normalIndex, hiddenIndex)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetadata().hasIndex(normalIndex), equalTo(true)); @@ -1916,14 +2201,16 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { // Verify that hidden indices can be restored with a non-star pattern { - RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin() - .prepareRestoreSnapshot(repoName, snapName) + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapName) .setWaitForCompletion(true) .setIndices("hid*") - .execute().get(); + .execute() + .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()) + ); assertThat(restoreSnapshotResponse.getRestoreInfo().indices(), containsInAnyOrder(hiddenIndex)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetadata().hasIndex(normalIndex), equalTo(false)); @@ -1934,14 +2221,15 @@ public void testHiddenIndicesIncludedInSnapshot() throws Exception { // Verify that hidden indices can be restored by fully specified name { - RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin() - .prepareRestoreSnapshot(repoName, snapName) + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(repoName, snapName) .setWaitForCompletion(true) .setIndices(dottedHiddenIndex) .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), - equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())); + assertThat( + restoreSnapshotResponse.getRestoreInfo().successfulShards(), + equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards()) + ); assertThat(restoreSnapshotResponse.getRestoreInfo().indices(), containsInAnyOrder(dottedHiddenIndex)); ClusterState clusterState = client.admin().cluster().prepareState().get().getState(); assertThat(clusterState.getMetadata().hasIndex(normalIndex), equalTo(false)); @@ -1959,21 +2247,27 @@ public void testIndexLatestFailuresIgnored() throws Exception { createFullSnapshot(repoName, "snapshot-1"); repository.setFailOnIndexLatest(false); createFullSnapshot(repoName, "snapshot-2"); - final long repoGenInIndexLatest = - Numbers.bytesToLong(new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB)))); + final long repoGenInIndexLatest = Numbers.bytesToLong( + new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB))) + ); assertEquals(getRepositoryData(repoName).getGenId(), repoGenInIndexLatest); - createRepository(repoName, "fs", Settings.builder() - .put("location", repoPath).put(BlobStoreRepository.SUPPORT_URL_REPO.getKey(), false)); + createRepository( + repoName, + "fs", + Settings.builder().put("location", repoPath).put(BlobStoreRepository.SUPPORT_URL_REPO.getKey(), false) + ); createFullSnapshot(repoName, "snapshot-3"); - final long repoGenInIndexLatest2 = - Numbers.bytesToLong(new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB)))); + final long repoGenInIndexLatest2 = Numbers.bytesToLong( + new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB))) + ); assertEquals("index.latest should not have been written to", repoGenInIndexLatest, repoGenInIndexLatest2); createRepository(repoName, "fs", repoPath); createFullSnapshot(repoName, "snapshot-4"); - final long repoGenInIndexLatest3 = - Numbers.bytesToLong(new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB)))); + final long repoGenInIndexLatest3 = Numbers.bytesToLong( + new BytesRef(Files.readAllBytes(repoPath.resolve(BlobStoreRepository.INDEX_LATEST_BLOB))) + ); assertEquals(getRepositoryData(repoName).getGenId(), repoGenInIndexLatest3); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotBrokenSettingsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotBrokenSettingsIT.java index b2a30c542751c..ba135a1ca4870 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotBrokenSettingsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotBrokenSettingsIT.java @@ -34,12 +34,27 @@ public void testExceptionWhenRestoringPersistentSettings() { internalCluster().startNodes(2); Client client = client(); - Consumer setSettingValue = value -> client.admin().cluster().prepareUpdateSettings().setPersistentSettings( - Settings.builder().put(BrokenSettingPlugin.BROKEN_SETTING.getKey(), value)).execute().actionGet(); - - Consumer assertSettingValue = value -> assertThat(client.admin().cluster().prepareState().setRoutingTable(false) - .setNodes(false).execute().actionGet().getState().getMetadata().persistentSettings() - .get(BrokenSettingPlugin.BROKEN_SETTING.getKey()), equalTo(value)); + Consumer setSettingValue = value -> client.admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(BrokenSettingPlugin.BROKEN_SETTING.getKey(), value)) + .execute() + .actionGet(); + + Consumer assertSettingValue = value -> assertThat( + client.admin() + .cluster() + .prepareState() + .setRoutingTable(false) + .setNodes(false) + .execute() + .actionGet() + .getState() + .getMetadata() + .persistentSettings() + .get(BrokenSettingPlugin.BROKEN_SETTING.getKey()), + equalTo(value) + ); logger.info("--> set test persistent setting"); setSettingValue.accept("new value"); @@ -55,9 +70,15 @@ public void testExceptionWhenRestoringPersistentSettings() { BrokenSettingPlugin.breakSetting(); logger.info("--> restore snapshot"); - final IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, - client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setRestoreGlobalState(true) - .setWaitForCompletion(true).execute()::actionGet); + final IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + client.admin() + .cluster() + .prepareRestoreSnapshot("test-repo", "test-snap") + .setRestoreGlobalState(true) + .setWaitForCompletion(true) + .execute()::actionGet + ); assertEquals(BrokenSettingPlugin.EXCEPTION.getMessage(), ex.getMessage()); assertSettingValue.accept("new value 2"); @@ -65,19 +86,17 @@ public void testExceptionWhenRestoringPersistentSettings() { public static class BrokenSettingPlugin extends Plugin { private static boolean breakSetting = false; - private static final IllegalArgumentException EXCEPTION = new IllegalArgumentException("this setting goes boom"); + private static final IllegalArgumentException EXCEPTION = new IllegalArgumentException("this setting goes boom"); static void breakSetting() { BrokenSettingPlugin.breakSetting = true; } - static final Setting BROKEN_SETTING = new Setting<>("setting.broken", "default", s->s, - s-> { - if ((s.equals("default") == false && breakSetting)) { - throw EXCEPTION; - } - }, - Setting.Property.NodeScope, Setting.Property.Dynamic); + static final Setting BROKEN_SETTING = new Setting<>("setting.broken", "default", s -> s, s -> { + if ((s.equals("default") == false && breakSetting)) { + throw EXCEPTION; + } + }, Setting.Property.NodeScope, Setting.Property.Dynamic); @Override public List> getSettings() { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotCustomPluginStateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotCustomPluginStateIT.java index bc8f34ddb6259..b23a7dc2c8e16 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotCustomPluginStateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotCustomPluginStateIT.java @@ -50,12 +50,15 @@ public void testIncludeGlobalState() throws Exception { boolean testPipeline = randomBoolean(); boolean testScript = (testTemplate == false && testPipeline == false) || randomBoolean(); // At least something should be stored - if(testTemplate) { + if (testTemplate) { logger.info("--> creating test template"); - assertThat(client().admin().indices() + assertThat( + client().admin() + .indices() .preparePutTemplate("test-template") .setPatterns(Collections.singletonList("te*")) - .setMapping(XContentFactory.jsonBuilder() + .setMapping( + XContentFactory.jsonBuilder() .startObject() .startObject("_doc") .startObject("properties") @@ -69,13 +72,18 @@ public void testIncludeGlobalState() throws Exception { .endObject() .endObject() .endObject() - .endObject()) - .get().isAcknowledged(), equalTo(true)); + .endObject() + ) + .get() + .isAcknowledged(), + equalTo(true) + ); } - if(testPipeline) { + if (testPipeline) { logger.info("--> creating test pipeline"); - BytesReference pipelineSource = BytesReference.bytes(jsonBuilder().startObject() + BytesReference pipelineSource = BytesReference.bytes( + jsonBuilder().startObject() .field("description", "my_pipeline") .startArray("processors") .startObject() @@ -83,39 +91,49 @@ public void testIncludeGlobalState() throws Exception { .endObject() .endObject() .endArray() - .endObject()); + .endObject() + ); assertAcked(clusterAdmin().preparePutPipeline("barbaz", pipelineSource, XContentType.JSON).get()); } - if(testScript) { + if (testScript) { logger.info("--> creating test script"); - assertAcked(clusterAdmin().preparePutStoredScript() + assertAcked( + clusterAdmin().preparePutStoredScript() .setId("foobar") - .setContent(new BytesArray( - "{\"script\": { \"lang\": \"" + MockScriptEngine.NAME + "\", \"source\": \"1\"} }"), XContentType.JSON)); + .setContent( + new BytesArray("{\"script\": { \"lang\": \"" + MockScriptEngine.NAME + "\", \"source\": \"1\"} }"), + XContentType.JSON + ) + ); } logger.info("--> snapshot without global state"); - CreateSnapshotResponse createSnapshotResponse = clusterAdmin() - .prepareCreateSnapshot("test-repo", "test-snap-no-global-state").setIndices().setIncludeGlobalState(false) - .setWaitForCompletion(true).get(); + CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-no-global-state") + .setIndices() + .setIncludeGlobalState(false) + .setWaitForCompletion(true) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), equalTo(0)); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(0)); assertThat(getSnapshot("test-repo", "test-snap-no-global-state").state(), equalTo(SnapshotState.SUCCESS)); SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo") - .addSnapshots("test-snap-no-global-state").get(); + .addSnapshots("test-snap-no-global-state") + .get(); assertThat(snapshotsStatusResponse.getSnapshots().size(), equalTo(1)); SnapshotStatus snapshotStatus = snapshotsStatusResponse.getSnapshots().get(0); assertThat(snapshotStatus.includeGlobalState(), equalTo(false)); logger.info("--> snapshot with global state"); createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-with-global-state") - .setIndices().setIncludeGlobalState(true).setWaitForCompletion(true).get(); + .setIndices() + .setIncludeGlobalState(true) + .setWaitForCompletion(true) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), equalTo(0)); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(0)); assertThat(getSnapshot("test-repo", "test-snap-with-global-state").state(), equalTo(SnapshotState.SUCCESS)); - snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo") - .addSnapshots("test-snap-with-global-state").get(); + snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo").addSnapshots("test-snap-with-global-state").get(); assertThat(snapshotsStatusResponse.getSnapshots().size(), equalTo(1)); snapshotStatus = snapshotsStatusResponse.getSnapshots().get(0); assertThat(snapshotStatus.includeGlobalState(), equalTo(true)); @@ -138,9 +156,11 @@ public void testIncludeGlobalState() throws Exception { } logger.info("--> try restoring cluster state from snapshot without global state"); - RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin() - .prepareRestoreSnapshot("test-repo", "test-snap-no-global-state") - .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); + RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-no-global-state") + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0)); logger.info("--> check that template wasn't restored"); @@ -149,7 +169,10 @@ public void testIncludeGlobalState() throws Exception { logger.info("--> restore cluster state"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-with-global-state") - .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0)); if (testTemplate) { @@ -174,10 +197,15 @@ public void testIncludeGlobalState() throws Exception { logger.info("--> snapshot without global state but with indices"); createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-no-global-state-with-index") - .setIndices("test-idx").setIncludeGlobalState(false).setWaitForCompletion(true).get(); + .setIndices("test-idx") + .setIncludeGlobalState(false) + .setWaitForCompletion(true) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); assertThat(getSnapshot("test-repo", "test-snap-no-global-state-with-index").state(), equalTo(SnapshotState.SUCCESS)); logger.info("--> delete global state and index "); @@ -198,7 +226,10 @@ public void testIncludeGlobalState() throws Exception { logger.info("--> try restoring index and cluster state from snapshot without global state"); restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-no-global-state-with-index") - .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet(); + .setWaitForCompletion(true) + .setRestoreGlobalState(true) + .execute() + .actionGet(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java index 3bb05e918a3b3..69d80cd4cefda 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotShardsServiceIT.java @@ -47,7 +47,9 @@ public void testRetryPostingSnapshotStatusMessages() throws Exception { logger.info("--> blocking repository"); String blockedNode = blockNodeWithIndex("test-repo", "test-index"); - dataNodeClient().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") + dataNodeClient().admin() + .cluster() + .prepareCreateSnapshot("test-repo", "test-snap") .setWaitForCompletion(false) .setIndices("test-index") .get(); @@ -68,7 +70,10 @@ public void testRetryPostingSnapshotStatusMessages() throws Exception { assertBusy(() -> { final Snapshot snapshot = new Snapshot("test-repo", snapshotId); List stages = snapshotShardsService.currentSnapshotShards(snapshot) - .values().stream().map(status -> status.asCopy().getStage()).collect(Collectors.toList()); + .values() + .stream() + .map(status -> status.asCopy().getStage()) + .collect(Collectors.toList()); assertThat(stages, hasSize(shards)); assertThat(stages, everyItem(equalTo(IndexShardSnapshotStatus.Stage.DONE))); }, 30L, TimeUnit.SECONDS); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java index a2c6f646b702c..e4c4267b45ed8 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java @@ -55,9 +55,10 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings)) - .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order - .build(); + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order + .build(); } public void testStatusApiConsistency() throws Exception { @@ -83,7 +84,8 @@ public void testStatusApiConsistency() throws Exception { assertThat(snapshotInfo.version(), equalTo(Version.CURRENT)); final List snapshotStatus = clusterAdmin().snapshotsStatus( - new SnapshotsStatusRequest("test-repo", new String[]{"test-snap"})).get().getSnapshots(); + new SnapshotsStatusRequest("test-repo", new String[] { "test-snap" }) + ).get().getSnapshots(); assertThat(snapshotStatus.size(), equalTo(1)); final SnapshotStatus snStatus = snapshotStatus.get(0); assertEquals(snStatus.getStats().getStartTime(), snapshotInfo.startTime()); @@ -108,8 +110,10 @@ public void testStatusAPICallInProgressSnapshot() throws Exception { logger.info("--> wait for data nodes to get blocked"); waitForBlockOnAnyDataNode("test-repo"); awaitNumberOfSnapshotsInProgress(1); - assertEquals(SnapshotsInProgress.State.STARTED, client().admin().cluster().prepareSnapshotStatus("test-repo") - .setSnapshots("test-snap").get().getSnapshots().get(0).getState()); + assertEquals( + SnapshotsInProgress.State.STARTED, + client().admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get().getSnapshots().get(0).getState() + ); logger.info("--> unblock all data nodes"); unblockAllDataNodes("test-repo"); @@ -127,8 +131,10 @@ public void testExceptionOnMissingSnapBlob() throws IOException { logger.info("--> delete snap-${uuid}.dat file for this snapshot to simulate concurrent delete"); IOUtils.rm(repoPath.resolve(BlobStoreRepository.SNAPSHOT_PREFIX + snapshotInfo.snapshotId().getUUID() + ".dat")); - GetSnapshotsResponse snapshotsResponse = client().admin().cluster() - .getSnapshots(new GetSnapshotsRequest(new String[] {"test-repo"}, new String[] {"test-snap"})).actionGet(); + GetSnapshotsResponse snapshotsResponse = client().admin() + .cluster() + .getSnapshots(new GetSnapshotsRequest(new String[] { "test-repo" }, new String[] { "test-snap" })) + .actionGet(); assertThat(snapshotsResponse.getFailedResponses().get("test-repo"), instanceOf(SnapshotMissingException.class)); } @@ -151,24 +157,35 @@ public void testExceptionOnMissingShardLevelSnapBlob() throws IOException { logger.info("--> delete shard-level snap-${uuid}.dat file for one shard in this snapshot to simulate concurrent delete"); final String indexRepoId = getRepositoryData("test-repo").resolveIndexId(snapshotInfo.indices().get(0)).getId(); - IOUtils.rm(repoPath.resolve("indices").resolve(indexRepoId).resolve("0").resolve( - BlobStoreRepository.SNAPSHOT_PREFIX + snapshotInfo.snapshotId().getUUID() + ".dat")); - - expectThrows(SnapshotMissingException.class, () -> client().admin().cluster() - .prepareSnapshotStatus("test-repo").setSnapshots("test-snap").execute().actionGet()); + IOUtils.rm( + repoPath.resolve("indices") + .resolve(indexRepoId) + .resolve("0") + .resolve(BlobStoreRepository.SNAPSHOT_PREFIX + snapshotInfo.snapshotId().getUUID() + ".dat") + ); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").execute().actionGet() + ); } public void testGetSnapshotsWithoutIndices() throws Exception { createRepository("test-repo", "fs"); logger.info("--> snapshot"); - final SnapshotInfo snapshotInfo = assertSuccessful(client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") - .setIndices().setWaitForCompletion(true).execute()); + final SnapshotInfo snapshotInfo = assertSuccessful( + client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setIndices().setWaitForCompletion(true).execute() + ); assertThat(snapshotInfo.totalShards(), is(0)); logger.info("--> verify that snapshot without index shows up in non-verbose listing"); - final List snapshotInfos = - client().admin().cluster().prepareGetSnapshots("test-repo").setVerbose(false).get().getSnapshots("test-repo"); + final List snapshotInfos = client().admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setVerbose(false) + .get() + .getSnapshots("test-repo"); assertThat(snapshotInfos, hasSize(1)); final SnapshotInfo found = snapshotInfos.get(0); assertThat(found.snapshotId(), is(snapshotInfo.snapshotId())); @@ -204,8 +221,12 @@ public void testCorrectCountsForDoneShards() throws Exception { final String snapshotOne = "snap-1"; // restarting a data node below so using a master client here - final ActionFuture responseSnapshotOne = internalCluster().masterClient().admin() - .cluster().prepareCreateSnapshot(repoName, snapshotOne).setWaitForCompletion(true).execute(); + final ActionFuture responseSnapshotOne = internalCluster().masterClient() + .admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotOne) + .setWaitForCompletion(true) + .execute(); assertBusy(() -> { final SnapshotStatus snapshotStatusOne = getSnapshotStatus(repoName, snapshotOne); @@ -215,15 +236,16 @@ public void testCorrectCountsForDoneShards() throws Exception { assertThat(snapshotShardState.getStats().getTotalSize(), greaterThan(0L)); }, 30L, TimeUnit.SECONDS); - final SnapshotStats snapshotShardStats = - stateFirstShard(getSnapshotStatus(repoName, snapshotOne), indexTwo).getStats(); + final SnapshotStats snapshotShardStats = stateFirstShard(getSnapshotStatus(repoName, snapshotOne), indexTwo).getStats(); final int totalFiles = snapshotShardStats.getTotalFileCount(); final long totalFileSize = snapshotShardStats.getTotalSize(); internalCluster().restartNode(dataNodeTwo); - final SnapshotIndexShardStatus snapshotShardStateAfterNodeRestart = - stateFirstShard(getSnapshotStatus(repoName, snapshotOne), indexTwo); + final SnapshotIndexShardStatus snapshotShardStateAfterNodeRestart = stateFirstShard( + getSnapshotStatus(repoName, snapshotOne), + indexTwo + ); assertThat(snapshotShardStateAfterNodeRestart.getStage(), is(SnapshotIndexShardStage.DONE)); assertThat(snapshotShardStateAfterNodeRestart.getStats().getTotalFileCount(), equalTo(totalFiles)); assertThat(snapshotShardStateAfterNodeRestart.getStats().getTotalSize(), equalTo(totalFileSize)); @@ -237,8 +259,11 @@ public void testCorrectCountsForDoneShards() throws Exception { blockDataNode(repoName, dataNodeTwo); final String snapshotTwo = "snap-2"; - final ActionFuture responseSnapshotTwo = - client().admin().cluster().prepareCreateSnapshot(repoName, snapshotTwo).setWaitForCompletion(true).execute(); + final ActionFuture responseSnapshotTwo = client().admin() + .cluster() + .prepareCreateSnapshot(repoName, snapshotTwo) + .setWaitForCompletion(true) + .execute(); waitForBlock(dataNodeTwo, repoName); @@ -272,10 +297,9 @@ public void testCorrectCountsForDoneShards() throws Exception { public void testGetSnapshotsNoRepos() { ensureGreen(); - GetSnapshotsResponse getSnapshotsResponse = clusterAdmin() - .prepareGetSnapshots(new String[]{"_all"}) - .setSnapshots(randomFrom("_all", "*")) - .get(); + GetSnapshotsResponse getSnapshotsResponse = clusterAdmin().prepareGetSnapshots(new String[] { "_all" }) + .setSnapshots(randomFrom("_all", "*")) + .get(); assertTrue(getSnapshotsResponse.getRepositories().isEmpty()); assertTrue(getSnapshotsResponse.getFailedResponses().isEmpty()); @@ -293,15 +317,21 @@ public void testGetSnapshotsMultipleRepos() throws Exception { final String indexName = "test-idx"; createIndexWithRandomDocs(indexName, 10); final int numberOfShards = IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.get( - client.admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName)); + client.admin().indices().prepareGetSettings(indexName).get().getIndexToSettings().get(indexName) + ); for (int repoIndex = 0; repoIndex < randomIntBetween(2, 5); repoIndex++) { final String repoName = "repo" + repoIndex; repoList.add(repoName); final Path repoPath = randomRepoPath(); logger.info("--> create repository with name " + repoName); - assertAcked(client.admin().cluster().preparePutRepository(repoName) - .setType("fs").setSettings(Settings.builder().put("location", repoPath).build())); + assertAcked( + client.admin() + .cluster() + .preparePutRepository(repoName) + .setType("fs") + .setSettings(Settings.builder().put("location", repoPath).build()) + ); List snapshotNames = new ArrayList<>(); repo2SnapshotNames.put(repoName, snapshotNames); @@ -315,11 +345,11 @@ public void testGetSnapshotsMultipleRepos() throws Exception { } logger.info("--> create snapshot with index {} and name {} in repository {}", snapshotIndex, snapshotName, repoName); CreateSnapshotResponse createSnapshotResponse = client.admin() - .cluster() - .prepareCreateSnapshot(repoName, snapshotName) - .setWaitForCompletion(true) - .setIndices(indexName) - .get(); + .cluster() + .prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true) + .setIndices(indexName) + .get(); final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); assertThat(snapshotInfo.successfulShards(), greaterThan(0)); assertTrue(snapshotInfo.indexSnapshotDetails().containsKey(indexName)); @@ -332,10 +362,11 @@ public void testGetSnapshotsMultipleRepos() throws Exception { } logger.info("--> get and verify snapshots"); - GetSnapshotsResponse getSnapshotsResponse = client.admin().cluster() - .prepareGetSnapshots(randomFrom(new String[]{"_all"}, new String[]{"repo*"}, repoList.toArray(new String[0]))) - .setSnapshots(randomFrom("_all", "*")) - .get(); + GetSnapshotsResponse getSnapshotsResponse = client.admin() + .cluster() + .prepareGetSnapshots(randomFrom(new String[] { "_all" }, new String[] { "repo*" }, repoList.toArray(new String[0]))) + .setSnapshots(randomFrom("_all", "*")) + .get(); for (Map.Entry> repo2Names : repo2SnapshotNames.entrySet()) { String repo = repo2Names.getKey(); @@ -345,23 +376,24 @@ public void testGetSnapshotsMultipleRepos() throws Exception { } logger.info("--> specify all snapshot names with ignoreUnavailable=false"); - GetSnapshotsResponse getSnapshotsResponse2 = client.admin().cluster() - .prepareGetSnapshots(randomFrom("_all", "repo*")) - .setIgnoreUnavailable(false) - .setSnapshots(snapshotList.toArray(new String[0])) - .get(); + GetSnapshotsResponse getSnapshotsResponse2 = client.admin() + .cluster() + .prepareGetSnapshots(randomFrom("_all", "repo*")) + .setIgnoreUnavailable(false) + .setSnapshots(snapshotList.toArray(new String[0])) + .get(); for (String repo : repoList) { expectThrows(SnapshotMissingException.class, () -> getSnapshotsResponse2.getSnapshots(repo)); } - logger.info("--> specify all snapshot names with ignoreUnavailable=true"); - GetSnapshotsResponse getSnapshotsResponse3 = client.admin().cluster() - .prepareGetSnapshots(randomFrom("_all", "repo*")) - .setIgnoreUnavailable(true) - .setSnapshots(snapshotList.toArray(new String[0])) - .get(); + GetSnapshotsResponse getSnapshotsResponse3 = client.admin() + .cluster() + .prepareGetSnapshots(randomFrom("_all", "repo*")) + .setIgnoreUnavailable(true) + .setSnapshots(snapshotList.toArray(new String[0])) + .get(); for (Map.Entry> repo2Names : repo2SnapshotNames.entrySet()) { String repo = repo2Names.getKey(); @@ -383,7 +415,10 @@ public void testGetSnapshotsWithSnapshotInProgress() throws Exception { waitForBlockOnAnyDataNode("test-repo"); awaitNumberOfSnapshotsInProgress(1); - GetSnapshotsResponse response1 = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap") + GetSnapshotsResponse response1 = client().admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setSnapshots("test-snap") .setIgnoreUnavailable(true) .get(); List snapshotInfoList = response1.getSnapshots("test-repo"); @@ -391,12 +426,18 @@ public void testGetSnapshotsWithSnapshotInProgress() throws Exception { assertEquals(SnapshotState.IN_PROGRESS, snapshotInfoList.get(0).state()); String notExistedSnapshotName = "snapshot_not_exist"; - GetSnapshotsResponse response2 = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots(notExistedSnapshotName) + GetSnapshotsResponse response2 = client().admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setSnapshots(notExistedSnapshotName) .setIgnoreUnavailable(true) .get(); assertEquals(0, response2.getSnapshots("test-repo").size()); - GetSnapshotsResponse response3 = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots(notExistedSnapshotName) + GetSnapshotsResponse response3 = client().admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setSnapshots(notExistedSnapshotName) .setIgnoreUnavailable(false) .get(); expectThrows(SnapshotMissingException.class, () -> response3.getSnapshots("test-repo")); @@ -418,8 +459,7 @@ public void testSnapshotStatusOnFailedSnapshot() throws Exception { ensureGreen(); indexRandomDocs("test-idx-good", randomIntBetween(1, 5)); - final SnapshotsStatusResponse snapshotsStatusResponse = - clusterAdmin().prepareSnapshotStatus(repoName).setSnapshots(snapshot).get(); + final SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus(repoName).setSnapshots(snapshot).get(); assertEquals(1, snapshotsStatusResponse.getSnapshots().size()); assertEquals(SnapshotsInProgress.State.FAILED, snapshotsStatusResponse.getSnapshots().get(0).getState()); } @@ -429,24 +469,33 @@ public void testGetSnapshotsRequest() throws Exception { final String indexName = "test-idx"; final Client client = client(); - createRepository(repositoryName, "mock", Settings.builder() - .put("location", randomRepoPath()).put("compress", false) - .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES).put("wait_after_unblock", 200)); + createRepository( + repositoryName, + "mock", + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + .put("wait_after_unblock", 200) + ); logger.info("--> get snapshots on an empty repository"); - expectThrows(SnapshotMissingException.class, () -> client.admin() + expectThrows( + SnapshotMissingException.class, + () -> client.admin() .cluster() .prepareGetSnapshots(repositoryName) .addSnapshots("non-existent-snapshot") .get() - .getSnapshots(repositoryName)); + .getSnapshots(repositoryName) + ); // with ignore unavailable set to true, should not throw an exception GetSnapshotsResponse getSnapshotsResponse = client.admin() - .cluster() - .prepareGetSnapshots(repositoryName) - .setIgnoreUnavailable(true) - .addSnapshots("non-existent-snapshot") - .get(); + .cluster() + .prepareGetSnapshots(repositoryName) + .setIgnoreUnavailable(true) + .addSnapshots("non-existent-snapshot") + .get(); assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(0)); logger.info("--> creating an index and indexing documents"); @@ -459,15 +508,18 @@ public void testGetSnapshotsRequest() throws Exception { // take initial snapshot with a block, making sure we only get 1 in-progress snapshot returned // block a node so the create snapshot operation can remain in progress final String initialBlockedNode = blockNodeWithIndex(repositoryName, indexName); - client.admin().cluster().prepareCreateSnapshot(repositoryName, "snap-on-empty-repo") - .setWaitForCompletion(false) - .setIndices(indexName) - .get(); + client.admin() + .cluster() + .prepareCreateSnapshot(repositoryName, "snap-on-empty-repo") + .setWaitForCompletion(false) + .setIndices(indexName) + .get(); waitForBlock(initialBlockedNode, repositoryName); // wait for block to kick in - getSnapshotsResponse = client.admin().cluster() - .prepareGetSnapshots("test-repo") - .setSnapshots(randomFrom("_all", "_current", "snap-on-*", "*-on-empty-repo", "snap-on-empty-repo")) - .get(); + getSnapshotsResponse = client.admin() + .cluster() + .prepareGetSnapshots("test-repo") + .setSnapshots(randomFrom("_all", "_current", "snap-on-*", "*-on-empty-repo", "snap-on-empty-repo")) + .get(); assertEquals(1, getSnapshotsResponse.getSnapshots("test-repo").size()); assertEquals("snap-on-empty-repo", getSnapshotsResponse.getSnapshots("test-repo").get(0).snapshotId().getName()); unblockNode(repositoryName, initialBlockedNode); // unblock node @@ -479,11 +531,11 @@ public void testGetSnapshotsRequest() throws Exception { for (int i = 0; i < numSnapshots - 1; i++) { final String snapshotName = randomAlphaOfLength(8).toLowerCase(Locale.ROOT); CreateSnapshotResponse createSnapshotResponse = client.admin() - .cluster() - .prepareCreateSnapshot(repositoryName, snapshotName) - .setWaitForCompletion(true) - .setIndices(indexName) - .get(); + .cluster() + .prepareCreateSnapshot(repositoryName, snapshotName) + .setWaitForCompletion(true) + .setIndices(indexName) + .get(); assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); snapshotNames[i] = snapshotName; } @@ -498,10 +550,12 @@ public void testGetSnapshotsRequest() throws Exception { snapshotNames[numSnapshots - 1] = inProgressSnapshot; // block a node so the create snapshot operation can remain in progress final String blockedNode = blockNodeWithIndex(repositoryName, indexName); - client.admin().cluster().prepareCreateSnapshot(repositoryName, inProgressSnapshot) - .setWaitForCompletion(false) - .setIndices(indexName) - .get(); + client.admin() + .cluster() + .prepareCreateSnapshot(repositoryName, inProgressSnapshot) + .setWaitForCompletion(false) + .setIndices(indexName) + .get(); waitForBlock(blockedNode, repositoryName); // wait for block to kick in logger.info("--> get all snapshots with a current in-progress"); @@ -516,45 +570,56 @@ public void testGetSnapshotsRequest() throws Exception { } else { snapshotsToGet.add("_all"); } - getSnapshotsResponse = client.admin().cluster() - .prepareGetSnapshots(repositoryName) - .setSnapshots(snapshotsToGet.toArray(Strings.EMPTY_ARRAY)) - .get(); + getSnapshotsResponse = client.admin() + .cluster() + .prepareGetSnapshots(repositoryName) + .setSnapshots(snapshotsToGet.toArray(Strings.EMPTY_ARRAY)) + .get(); List sortedNames = Arrays.asList(snapshotNames); Collections.sort(sortedNames); assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots)); - assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream() + assertThat( + getSnapshotsResponse.getSnapshots(repositoryName) + .stream() .map(s -> s.snapshotId().getName()) .sorted() - .collect(Collectors.toList()), equalTo(sortedNames)); + .collect(Collectors.toList()), + equalTo(sortedNames) + ); - getSnapshotsResponse = client.admin().cluster() - .prepareGetSnapshots(repositoryName) - .addSnapshots(snapshotNames) - .get(); + getSnapshotsResponse = client.admin().cluster().prepareGetSnapshots(repositoryName).addSnapshots(snapshotNames).get(); sortedNames = Arrays.asList(snapshotNames); Collections.sort(sortedNames); assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots)); - assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream() + assertThat( + getSnapshotsResponse.getSnapshots(repositoryName) + .stream() .map(s -> s.snapshotId().getName()) .sorted() - .collect(Collectors.toList()), equalTo(sortedNames)); + .collect(Collectors.toList()), + equalTo(sortedNames) + ); logger.info("--> make sure duplicates are not returned in the response"); String regexName = snapshotNames[randomIntBetween(0, numSnapshots - 1)]; final int splitPos = regexName.length() / 2; final String firstRegex = regexName.substring(0, splitPos) + "*"; final String secondRegex = "*" + regexName.substring(splitPos); - getSnapshotsResponse = client.admin().cluster() - .prepareGetSnapshots(repositoryName) - .addSnapshots(snapshotNames) - .addSnapshots(firstRegex, secondRegex) - .get(); + getSnapshotsResponse = client.admin() + .cluster() + .prepareGetSnapshots(repositoryName) + .addSnapshots(snapshotNames) + .addSnapshots(firstRegex, secondRegex) + .get(); assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots)); - assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream() + assertThat( + getSnapshotsResponse.getSnapshots(repositoryName) + .stream() .map(s -> s.snapshotId().getName()) .sorted() - .collect(Collectors.toList()), equalTo(sortedNames)); + .collect(Collectors.toList()), + equalTo(sortedNames) + ); unblockNode(repositoryName, blockedNode); // unblock node awaitNoMoreRunningOperations(); @@ -598,8 +663,7 @@ private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotS private static SnapshotStatus getSnapshotStatus(String repoName, String snapshotName) { try { - return client().admin().cluster().prepareSnapshotStatus(repoName).setSnapshots(snapshotName) - .get().getSnapshots().get(0); + return client().admin().cluster().prepareSnapshotStatus(repoName).setSnapshots(snapshotName).get().getSnapshots().get(0); } catch (SnapshotMissingException e) { throw new AssertionError(e); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java index 5354ca5b189ee..7b6abf0ec2b84 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java @@ -114,8 +114,10 @@ public void testSnapshotWithoutGlobalState() { // check snapshot info for for which clusterAdmin().prepareGetRepositories(REPO_NAME).get(); - Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME).get() - .getSnapshots(REPO_NAME).stream() + Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME) + .get() + .getSnapshots(REPO_NAME) + .stream() .map(SnapshotInfo::indices) .flatMap(Collection::stream) .collect(Collectors.toSet()); @@ -189,15 +191,17 @@ public void testDefaultRestoreOnlyRegularIndices() { assertAcked(cluster().client().admin().indices().prepareDelete(regularIndex)); // restore indices by feature, with only the regular index named explicitly - SnapshotRestoreException exception = expectThrows(SnapshotRestoreException.class, - () -> clusterAdmin().prepareRestoreSnapshot(REPO_NAME, "test-snap") - .setWaitForCompletion(true) - .get()); + SnapshotRestoreException exception = expectThrows( + SnapshotRestoreException.class, + () -> clusterAdmin().prepareRestoreSnapshot(REPO_NAME, "test-snap").setWaitForCompletion(true).get() + ); - assertThat(exception.getMessage(), containsString( - "cannot restore index [" + - SystemIndexTestPlugin.SYSTEM_INDEX_NAME - + "] because an open index with same name already exists")); + assertThat( + exception.getMessage(), + containsString( + "cannot restore index [" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "] because an open index with same name already exists" + ) + ); } /** @@ -268,8 +272,10 @@ public void testSnapshotAndRestoreAssociatedIndices() { assertSnapshotSuccess(createSnapshotResponse); // verify the correctness of the snapshot - Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME).get() - .getSnapshots(REPO_NAME).stream() + Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME) + .get() + .getSnapshots(REPO_NAME) + .stream() .map(SnapshotInfo::indices) .flatMap(Collection::stream) .collect(Collectors.toSet()); @@ -321,10 +327,13 @@ public void testRestoreFeatureNotInSnapshot() { () -> clusterAdmin().prepareRestoreSnapshot(REPO_NAME, "test-snap") .setWaitForCompletion(true) .setFeatureStates("SystemIndexTestPlugin", fakeFeatureStateName) - .get()); + .get() + ); - assertThat(exception.getMessage(), - containsString("requested feature states [[" + fakeFeatureStateName + "]] are not present in snapshot")); + assertThat( + exception.getMessage(), + containsString("requested feature states [[" + fakeFeatureStateName + "]] are not present in snapshot") + ); } /** @@ -351,11 +360,14 @@ public void testRestoringSystemIndexByNameIsDeprecated() throws IllegalAccessExc MockLogAppender mockLogAppender = new MockLogAppender(); Loggers.addAppender(LogManager.getLogger("org.elasticsearch.deprecation.snapshots.RestoreService"), mockLogAppender); mockLogAppender.start(); - mockLogAppender.addExpectation(new MockLogAppender.SeenEventExpectation( - "restore-system-index-from-snapshot", - "org.elasticsearch.deprecation.snapshots.RestoreService", - DeprecationLogger.DEPRECATION, - "Restoring system indices by name is deprecated. Use feature states instead. System indices: [.test-system-idx]")); + mockLogAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "restore-system-index-from-snapshot", + "org.elasticsearch.deprecation.snapshots.RestoreService", + DeprecationLogger.DEPRECATION, + "Restoring system indices by name is deprecated. Use feature states instead. System indices: [.test-system-idx]" + ) + ); // restore system index by name, rather than feature state RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot(REPO_NAME, "test-snap") @@ -475,7 +487,7 @@ public void testRestoreSystemIndicesAsGlobalStateWithEmptyListOfFeatureStates() .setIndices(regularIndex) .setWaitForCompletion(true) .setRestoreGlobalState(true) - .setFeatureStates(new String[]{ randomFrom("none", "NONE") }) + .setFeatureStates(new String[] { randomFrom("none", "NONE") }) .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0)); @@ -507,11 +519,18 @@ public void testRestoreSystemIndicesAsGlobalStateWithEmptyListOfFeatureStatesNoI () -> clusterAdmin().prepareRestoreSnapshot(REPO_NAME, "test-snap") .setWaitForCompletion(true) .setRestoreGlobalState(true) - .setFeatureStates(new String[]{ randomFrom("none", "NONE") }) - .get()); + .setFeatureStates(new String[] { randomFrom("none", "NONE") }) + .get() + ); - assertThat(exception.getMessage(), containsString("cannot restore index [" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME - + "] because an open index with same name already exists in the cluster.")); + assertThat( + exception.getMessage(), + containsString( + "cannot restore index [" + + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + + "] because an open index with same name already exists in the cluster." + ) + ); } /** @@ -584,9 +603,14 @@ public void testSystemIndexAliasesAreAlwaysRestored() { // And make sure they both have aliases final String systemIndexAlias = SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "-alias"; - assertAcked(client().admin().indices().prepareAliases() - .addAlias(systemIndexName, systemIndexAlias) - .addAlias(regularIndex, regularAlias).get()); + assertAcked( + client().admin() + .indices() + .prepareAliases() + .addAlias(systemIndexName, systemIndexAlias) + .addAlias(regularIndex, regularAlias) + .get() + ); // run a snapshot including global state CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot(REPO_NAME, "test-snap") @@ -637,8 +661,13 @@ public void testNoneFeatureStateMustBeAlone() { .setFeatureStates("SystemIndexTestPlugin", "none", "AnotherSystemIndexTestPlugin") .get() ); - assertThat(createEx.getMessage(), equalTo("the feature_states value [none] indicates that no feature states should be " + - "snapshotted, but other feature states were requested: [SystemIndexTestPlugin, none, AnotherSystemIndexTestPlugin]")); + assertThat( + createEx.getMessage(), + equalTo( + "the feature_states value [none] indicates that no feature states should be " + + "snapshotted, but other feature states were requested: [SystemIndexTestPlugin, none, AnotherSystemIndexTestPlugin]" + ) + ); // create a successful snapshot with global state/all features CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot(REPO_NAME, "test-snap") @@ -657,7 +686,8 @@ public void testNoneFeatureStateMustBeAlone() { ); assertThat( restoreEx.getMessage(), - allOf( // the order of the requested feature states is non-deterministic so just check that it includes most of the right stuff + allOf( + // the order of the requested feature states is non-deterministic so just check that it includes most of the right stuff containsString( "the feature_states value [none] indicates that no feature states should be restored, but other feature states were " + "requested:" @@ -687,8 +717,10 @@ public void testNoneFeatureStateOnCreation() { assertSnapshotSuccess(createSnapshotResponse); // Verify that the system index was not included - Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME).get() - .getSnapshots(REPO_NAME).stream() + Set snapshottedIndices = clusterAdmin().prepareGetSnapshots(REPO_NAME) + .get() + .getSnapshots(REPO_NAME) + .stream() .map(SnapshotInfo::indices) .flatMap(Collection::stream) .collect(Collectors.toSet()); @@ -784,8 +816,11 @@ public void testPartialSnapshotsOfSystemIndexRemovesFeatureState() throws Except // Stop a random data node so we lose a shard from the partial index internalCluster().stopRandomDataNode(); - assertBusy(() -> assertEquals(ClusterHealthStatus.RED, client().admin().cluster().prepareHealth().get().getStatus()), - 30, TimeUnit.SECONDS); + assertBusy( + () -> assertEquals(ClusterHealthStatus.RED, client().admin().cluster().prepareHealth().get().getStatus()), + 30, + TimeUnit.SECONDS + ); // Get ready to block blockMasterFromFinalizingSnapshotOnIndexFile(REPO_NAME); @@ -803,12 +838,16 @@ public void testPartialSnapshotsOfSystemIndexRemovesFeatureState() throws Except // Now get the snapshot and do our checks assertBusy(() -> { - GetSnapshotsResponse snapshotsStatusResponse = client().admin().cluster() - .prepareGetSnapshots(REPO_NAME).setSnapshots(partialSnapName).get(); + GetSnapshotsResponse snapshotsStatusResponse = client().admin() + .cluster() + .prepareGetSnapshots(REPO_NAME) + .setSnapshots(partialSnapName) + .get(); SnapshotInfo snapshotInfo = snapshotsStatusResponse.getSnapshots(REPO_NAME).get(0); assertNotNull(snapshotInfo); assertThat(snapshotInfo.failedShards(), lessThan(snapshotInfo.totalShards())); - List statesInSnapshot = snapshotInfo.featureStates().stream() + List statesInSnapshot = snapshotInfo.featureStates() + .stream() .map(SnapshotFeatureInfo::getPluginName) .collect(Collectors.toList()); assertThat(statesInSnapshot, not(hasItem((new SystemIndexTestPlugin()).getFeatureName()))); @@ -875,7 +914,8 @@ public void testParallelIndexDeleteRemovesFeatureState() throws Exception { SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); assertNotNull(snapshotInfo); assertThat(snapshotInfo.indices(), not(hasItem(indexToBeDeleted))); - List statesInSnapshot = snapshotInfo.featureStates().stream() + List statesInSnapshot = snapshotInfo.featureStates() + .stream() .map(SnapshotFeatureInfo::getPluginName) .collect(Collectors.toList()); assertThat(statesInSnapshot, not(hasItem((new SystemIndexTestPlugin()).getFeatureName()))); @@ -884,8 +924,10 @@ public void testParallelIndexDeleteRemovesFeatureState() throws Exception { private void assertSnapshotSuccess(CreateSnapshotResponse createSnapshotResponse) { assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); } private long getDocCount(String indexName) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryRequestBuilder.java index 3fd3b0bedb19e..31dca81ce2989 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryRequestBuilder.java @@ -11,12 +11,12 @@ import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; -public class CleanupRepositoryRequestBuilder extends MasterNodeOperationRequestBuilder { +public class CleanupRepositoryRequestBuilder extends MasterNodeOperationRequestBuilder< + CleanupRepositoryRequest, + CleanupRepositoryResponse, + CleanupRepositoryRequestBuilder> { - public CleanupRepositoryRequestBuilder(ElasticsearchClient client, ActionType action, - String repository) { + public CleanupRepositoryRequestBuilder(ElasticsearchClient client, ActionType action, String repository) { super(client, action, new CleanupRepositoryRequest(repository)); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryResponse.java index 2f1e352f77d06..eb9d4397c7452 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/CleanupRepositoryResponse.java @@ -21,18 +21,23 @@ public final class CleanupRepositoryResponse extends ActionResponse implements ToXContentObject { - private static final ObjectParser PARSER = - new ObjectParser<>(CleanupRepositoryResponse.class.getName(), true, CleanupRepositoryResponse::new); + private static final ObjectParser PARSER = new ObjectParser<>( + CleanupRepositoryResponse.class.getName(), + true, + CleanupRepositoryResponse::new + ); static { - PARSER.declareObject((response, cleanupResult) -> response.result = cleanupResult, - RepositoryCleanupResult.PARSER, new ParseField("results")); + PARSER.declareObject( + (response, cleanupResult) -> response.result = cleanupResult, + RepositoryCleanupResult.PARSER, + new ParseField("results") + ); } private RepositoryCleanupResult result; - public CleanupRepositoryResponse() { - } + public CleanupRepositoryResponse() {} public CleanupRepositoryResponse(RepositoryCleanupResult result) { this.result = result; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java index 419507cbdf5ab..cf1f8f419d110 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/cleanup/TransportCleanupRepositoryAction.java @@ -56,8 +56,7 @@ * and any delete/write operations. TODO: This will not work if we also want to clean up at the shard level as those will involve writes * as well as deletes. */ -public final class TransportCleanupRepositoryAction extends TransportMasterNodeAction { +public final class TransportCleanupRepositoryAction extends TransportMasterNodeAction { private static final Logger logger = LogManager.getLogger(TransportCleanupRepositoryAction.class); @@ -66,12 +65,26 @@ public final class TransportCleanupRepositoryAction extends TransportMasterNodeA private final SnapshotsService snapshotsService; @Inject - public TransportCleanupRepositoryAction(TransportService transportService, ClusterService clusterService, - RepositoriesService repositoriesService, SnapshotsService snapshotsService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(CleanupRepositoryAction.NAME, transportService, clusterService, threadPool, actionFilters, - CleanupRepositoryRequest::new, indexNameExpressionResolver, CleanupRepositoryResponse::new, ThreadPool.Names.SAME); + public TransportCleanupRepositoryAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService, + SnapshotsService snapshotsService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + CleanupRepositoryAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + CleanupRepositoryRequest::new, + indexNameExpressionResolver, + CleanupRepositoryResponse::new, + ThreadPool.Names.SAME + ); this.repositoriesService = repositoriesService; this.snapshotsService = snapshotsService; // We add a state applier that will remove any dangling repository cleanup actions on master failover. @@ -85,12 +98,13 @@ public TransportCleanupRepositoryAction(TransportService transportService, Clust private static void addClusterStateApplier(ClusterService clusterService) { clusterService.addStateApplier(event -> { if (event.localNodeMaster() && event.previousState().nodes().isLocalNodeElectedMaster() == false) { - final RepositoryCleanupInProgress repositoryCleanupInProgress = - event.state().custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + final RepositoryCleanupInProgress repositoryCleanupInProgress = event.state() + .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); if (repositoryCleanupInProgress.hasCleanupInProgress() == false) { return; } - clusterService.submitStateUpdateTask("clean up repository cleanup task after master failover", + clusterService.submitStateUpdateTask( + "clean up repository cleanup task after master failover", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { @@ -104,10 +118,10 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS @Override public void onFailure(String source, Exception e) { - logger.warn( - "Failed to remove repository cleanup task [{}] from cluster state", repositoryCleanupInProgress); + logger.warn("Failed to remove repository cleanup task [{}] from cluster state", repositoryCleanupInProgress); } - }); + } + ); } }); } @@ -119,8 +133,12 @@ private static ClusterState removeInProgressCleanup(final ClusterState currentSt } @Override - protected void masterOperation(Task task, CleanupRepositoryRequest request, ClusterState state, - ActionListener listener) { + protected void masterOperation( + Task task, + CleanupRepositoryRequest request, + ClusterState state, + ActionListener listener + ) { cleanupRepo(request.name(), listener.map(CleanupRepositoryResponse::new)); } @@ -147,34 +165,54 @@ private void cleanupRepo(String repositoryName, ActionListener { final long repositoryStateId = repositoryData.getGenId(); logger.info("Running cleanup operations on repository [{}][{}]", repositoryName, repositoryStateId); - clusterService.submitStateUpdateTask("cleanup repository [" + repositoryName + "][" + repositoryStateId + ']', + clusterService.submitStateUpdateTask( + "cleanup repository [" + repositoryName + "][" + repositoryStateId + ']', new ClusterStateUpdateTask() { private boolean startedCleanup = false; @Override public ClusterState execute(ClusterState currentState) { - final RepositoryCleanupInProgress repositoryCleanupInProgress = - currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress.EMPTY + ); if (repositoryCleanupInProgress.hasCleanupInProgress()) { throw new IllegalStateException( - "Cannot cleanup [" + repositoryName + "] - a repository cleanup is already in-progress in [" - + repositoryCleanupInProgress + "]"); + "Cannot cleanup [" + + repositoryName + + "] - a repository cleanup is already in-progress in [" + + repositoryCleanupInProgress + + "]" + ); } - final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); if (deletionsInProgress.hasDeletionsInProgress()) { - throw new IllegalStateException("Cannot cleanup [" + repositoryName - + "] - a snapshot is currently being deleted in [" + deletionsInProgress + "]"); + throw new IllegalStateException( + "Cannot cleanup [" + + repositoryName + + "] - a snapshot is currently being deleted in [" + + deletionsInProgress + + "]" + ); } SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); if (snapshots.entries().isEmpty() == false) { throw new IllegalStateException( - "Cannot cleanup [" + repositoryName + "] - a snapshot is currently running in [" + snapshots + "]"); + "Cannot cleanup [" + repositoryName + "] - a snapshot is currently running in [" + snapshots + "]" + ); } - return ClusterState.builder(currentState).putCustom(RepositoryCleanupInProgress.TYPE, - new RepositoryCleanupInProgress( - List.of(RepositoryCleanupInProgress.startedEntry(repositoryName, repositoryStateId)))).build(); + return ClusterState.builder(currentState) + .putCustom( + RepositoryCleanupInProgress.TYPE, + new RepositoryCleanupInProgress( + List.of(RepositoryCleanupInProgress.startedEntry(repositoryName, repositoryStateId)) + ) + ) + .build(); } @Override @@ -186,21 +224,31 @@ public void onFailure(String source, Exception e) { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { startedCleanup = true; logger.debug("Initialized repository cleanup in cluster state for [{}][{}]", repositoryName, repositoryStateId); - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(listener, - l -> blobStoreRepository.cleanup( - repositoryStateId, - snapshotsService.minCompatibleVersion( - newState.nodes().getMinNodeVersion(), repositoryData, null), - ActionListener.wrap(result -> after(null, result), e -> after(e, null))) - )); + threadPool.executor(ThreadPool.Names.SNAPSHOT) + .execute( + ActionRunnable.wrap( + listener, + l -> blobStoreRepository.cleanup( + repositoryStateId, + snapshotsService.minCompatibleVersion(newState.nodes().getMinNodeVersion(), repositoryData, null), + ActionListener.wrap(result -> after(null, result), e -> after(e, null)) + ) + ) + ); } private void after(@Nullable Exception failure, @Nullable RepositoryCleanupResult result) { if (failure == null) { logger.debug("Finished repository cleanup operations on [{}][{}]", repositoryName, repositoryStateId); } else { - logger.debug(() -> new ParameterizedMessage( - "Failed to finish repository cleanup operations on [{}][{}]", repositoryName, repositoryStateId), failure); + logger.debug( + () -> new ParameterizedMessage( + "Failed to finish repository cleanup operations on [{}][{}]", + repositoryName, + repositoryStateId + ), + failure + ); } assert failure != null || result != null; if (startedCleanup == false) { @@ -221,27 +269,40 @@ public void onFailure(String source, Exception e) { if (failure != null) { e.addSuppressed(failure); } - logger.warn(() -> - new ParameterizedMessage("[{}] failed to remove repository cleanup task", repositoryName), e); + logger.warn( + () -> new ParameterizedMessage("[{}] failed to remove repository cleanup task", repositoryName), + e + ); listener.onFailure(e); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { if (failure == null) { - logger.info("Done with repository cleanup on [{}][{}] with result [{}]", - repositoryName, repositoryStateId, result); + logger.info( + "Done with repository cleanup on [{}][{}] with result [{}]", + repositoryName, + repositoryStateId, + result + ); listener.onResponse(result); } else { - logger.warn(() -> new ParameterizedMessage( - "Failed to run repository cleanup operations on [{}][{}]", - repositoryName, repositoryStateId), failure); + logger.warn( + () -> new ParameterizedMessage( + "Failed to run repository cleanup operations on [{}][{}]", + repositoryName, + repositoryStateId + ), + failure + ); listener.onFailure(failure); } } - }); + } + ); } - }); + } + ); }, listener::onFailure); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java index 6ed48cbd235f6..590460e9025b6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryAction.java @@ -24,4 +24,3 @@ private DeleteRepositoryAction() { } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequest.java index 0e61eacbdcb4d..27446da23d310 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequest.java @@ -31,8 +31,7 @@ public DeleteRepositoryRequest(StreamInput in) throws IOException { name = in.readString(); } - public DeleteRepositoryRequest() { - } + public DeleteRepositoryRequest() {} /** * Constructs a new unregister repository request with the provided name. diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequestBuilder.java index 1570138eb8845..d2c611172cab8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryRequestBuilder.java @@ -15,8 +15,10 @@ /** * Builder for unregister repository request */ -public class DeleteRepositoryRequestBuilder - extends AcknowledgedRequestBuilder { +public class DeleteRepositoryRequestBuilder extends AcknowledgedRequestBuilder< + DeleteRepositoryRequest, + AcknowledgedResponse, + DeleteRepositoryRequestBuilder> { /** * Constructs unregister repository request builder diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java index bdca243976422..7cde594f61065 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/TransportDeleteRepositoryAction.java @@ -31,11 +31,24 @@ public class TransportDeleteRepositoryAction extends AcknowledgedTransportMaster private final RepositoriesService repositoriesService; @Inject - public TransportDeleteRepositoryAction(TransportService transportService, ClusterService clusterService, - RepositoriesService repositoriesService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(DeleteRepositoryAction.NAME, transportService, clusterService, threadPool, actionFilters, - DeleteRepositoryRequest::new, indexNameExpressionResolver, ThreadPool.Names.SAME); + public TransportDeleteRepositoryAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + DeleteRepositoryAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + DeleteRepositoryRequest::new, + indexNameExpressionResolver, + ThreadPool.Names.SAME + ); this.repositoriesService = repositoriesService; } @@ -45,9 +58,15 @@ protected ClusterBlockException checkBlock(DeleteRepositoryRequest request, Clus } @Override - protected void masterOperation(Task task, final DeleteRepositoryRequest request, ClusterState state, - final ActionListener listener) { - repositoriesService.unregisterRepository(request, - listener.map(unregisterRepositoryResponse -> AcknowledgedResponse.of(unregisterRepositoryResponse.isAcknowledged()))); + protected void masterOperation( + Task task, + final DeleteRepositoryRequest request, + ClusterState state, + final ActionListener listener + ) { + repositoriesService.unregisterRepository( + request, + listener.map(unregisterRepositoryResponse -> AcknowledgedResponse.of(unregisterRepositoryResponse.isAcknowledged())) + ); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java index e1751b04af079..3411f29819f3c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesAction.java @@ -23,4 +23,3 @@ private GetRepositoriesAction() { } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java index 2741d7d92a7c7..c4fd56ac6ad4d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java @@ -25,8 +25,7 @@ public class GetRepositoriesRequest extends MasterNodeReadRequest { +public class GetRepositoriesRequestBuilder extends MasterNodeReadOperationRequestBuilder< + GetRepositoriesRequest, + GetRepositoriesResponse, + GetRepositoriesRequestBuilder> { /** * Creates new get repository request builder diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java index 4a4da998e7c23..93ecc7e7f5da6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java @@ -47,7 +47,6 @@ public List repositories() { return repositories.repositories(); } - @Override public void writeTo(StreamOutput out) throws IOException { repositories.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java index dd059ccdb5c9a..a6c9f9c717b96 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java @@ -36,11 +36,24 @@ public class TransportGetRepositoriesAction extends TransportMasterNodeReadAction { @Inject - public TransportGetRepositoriesAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(GetRepositoriesAction.NAME, transportService, clusterService, threadPool, actionFilters, - GetRepositoriesRequest::new, indexNameExpressionResolver, GetRepositoriesResponse::new, ThreadPool.Names.SAME); + public TransportGetRepositoriesAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + GetRepositoriesAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + GetRepositoriesRequest::new, + indexNameExpressionResolver, + GetRepositoriesResponse::new, + ThreadPool.Names.SAME + ); } @Override @@ -49,8 +62,12 @@ protected ClusterBlockException checkBlock(GetRepositoriesRequest request, Clust } @Override - protected void masterOperation(Task task, final GetRepositoriesRequest request, ClusterState state, - final ActionListener listener) { + protected void masterOperation( + Task task, + final GetRepositoriesRequest request, + ClusterState state, + final ActionListener listener + ) { listener.onResponse(new GetRepositoriesResponse(new RepositoriesMetadata(getRepositories(state, request.repositories())))); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java index a59056599fdc3..3ac2134afef83 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryAction.java @@ -24,4 +24,3 @@ private PutRepositoryAction() { } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequest.java index 8b19956766a90..57cfe29b7b5b1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequest.java @@ -49,8 +49,7 @@ public PutRepositoryRequest(StreamInput in) throws IOException { verify = in.readBoolean(); } - public PutRepositoryRequest() { - } + public PutRepositoryRequest() {} /** * Constructs a new put repository request with the provided name. diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequestBuilder.java index 8cc56069ab498..c41e1e0406cd8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/PutRepositoryRequestBuilder.java @@ -19,8 +19,10 @@ /** * Register repository request builder */ -public class PutRepositoryRequestBuilder - extends AcknowledgedRequestBuilder { +public class PutRepositoryRequestBuilder extends AcknowledgedRequestBuilder< + PutRepositoryRequest, + AcknowledgedResponse, + PutRepositoryRequestBuilder> { /** * Constructs register repository request diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java index 815f59827e2b5..833bbb68b4a3c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/put/TransportPutRepositoryAction.java @@ -31,11 +31,24 @@ public class TransportPutRepositoryAction extends AcknowledgedTransportMasterNod private final RepositoriesService repositoriesService; @Inject - public TransportPutRepositoryAction(TransportService transportService, ClusterService clusterService, - RepositoriesService repositoriesService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(PutRepositoryAction.NAME, transportService, clusterService, threadPool, actionFilters, - PutRepositoryRequest::new, indexNameExpressionResolver, ThreadPool.Names.SAME); + public TransportPutRepositoryAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + PutRepositoryAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + PutRepositoryRequest::new, + indexNameExpressionResolver, + ThreadPool.Names.SAME + ); this.repositoriesService = repositoriesService; } @@ -45,8 +58,12 @@ protected ClusterBlockException checkBlock(PutRepositoryRequest request, Cluster } @Override - protected void masterOperation(Task task, final PutRepositoryRequest request, ClusterState state, - final ActionListener listener) { + protected void masterOperation( + Task task, + final PutRepositoryRequest request, + ClusterState state, + final ActionListener listener + ) { repositoriesService.registerRepository(request, listener.map(response -> AcknowledgedResponse.of(response.isAcknowledged()))); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java index 3be7c82da63c6..4335dea3861e3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/TransportVerifyRepositoryAction.java @@ -26,18 +26,30 @@ /** * Transport action for verifying repository operation */ -public class TransportVerifyRepositoryAction extends - TransportMasterNodeAction { +public class TransportVerifyRepositoryAction extends TransportMasterNodeAction { private final RepositoriesService repositoriesService; - @Inject - public TransportVerifyRepositoryAction(TransportService transportService, ClusterService clusterService, - RepositoriesService repositoriesService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(VerifyRepositoryAction.NAME, transportService, clusterService, threadPool, actionFilters, - VerifyRepositoryRequest::new, indexNameExpressionResolver, VerifyRepositoryResponse::new, ThreadPool.Names.SAME); + public TransportVerifyRepositoryAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + VerifyRepositoryAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + VerifyRepositoryRequest::new, + indexNameExpressionResolver, + VerifyRepositoryResponse::new, + ThreadPool.Names.SAME + ); this.repositoriesService = repositoriesService; } @@ -47,9 +59,15 @@ protected ClusterBlockException checkBlock(VerifyRepositoryRequest request, Clus } @Override - protected void masterOperation(Task task, final VerifyRepositoryRequest request, ClusterState state, - final ActionListener listener) { - repositoriesService.verifyRepository(request.name(), listener.map( - verifyResponse -> new VerifyRepositoryResponse(verifyResponse.toArray(new DiscoveryNode[0])))); + protected void masterOperation( + Task task, + final VerifyRepositoryRequest request, + ClusterState state, + final ActionListener listener + ) { + repositoriesService.verifyRepository( + request.name(), + listener.map(verifyResponse -> new VerifyRepositoryResponse(verifyResponse.toArray(new DiscoveryNode[0]))) + ); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java index 816e5249ac39a..0daaae79592f1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryAction.java @@ -22,4 +22,3 @@ private VerifyRepositoryAction() { super(NAME, VerifyRepositoryResponse::new); } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequest.java index 314685507fa4a..621528c903119 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequest.java @@ -29,8 +29,7 @@ public VerifyRepositoryRequest(StreamInput in) throws IOException { name = in.readString(); } - public VerifyRepositoryRequest() { - } + public VerifyRepositoryRequest() {} /** * Constructs a new unregister repository request with the provided name. diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequestBuilder.java index b2e74f75e5d69..5e6840e8c0339 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryRequestBuilder.java @@ -14,8 +14,10 @@ /** * Builder for verify repository request */ -public class VerifyRepositoryRequestBuilder - extends MasterNodeOperationRequestBuilder { +public class VerifyRepositoryRequestBuilder extends MasterNodeOperationRequestBuilder< + VerifyRepositoryRequest, + VerifyRepositoryResponse, + VerifyRepositoryRequestBuilder> { /** * Constructs unregister repository request builder diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java index be332834c2ae2..c5a2582b81cb7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java @@ -45,7 +45,9 @@ public static class NodeView implements Writeable, ToXContentObject { final String nodeId; String name; - public NodeView(String nodeId) { this.nodeId = nodeId; } + public NodeView(String nodeId) { + this.nodeId = nodeId; + } public NodeView(String nodeId, String name) { this(nodeId); @@ -62,11 +64,17 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(name); } - void setName(String name) { this.name = name; } + void setName(String name) { + this.name = name; + } - public String getName() { return name; } + public String getName() { + return name; + } - public String getNodeId() { return nodeId; } + public String getNodeId() { + return nodeId; + } public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(nodeId); @@ -86,8 +94,7 @@ public boolean equals(Object obj) { return false; } NodeView other = (NodeView) obj; - return Objects.equals(nodeId, other.nodeId) && - Objects.equals(name, other.name); + return Objects.equals(nodeId, other.nodeId) && Objects.equals(name, other.name); } @Override @@ -98,8 +105,11 @@ public int hashCode() { private List nodes; - private static final ObjectParser PARSER = - new ObjectParser<>(VerifyRepositoryResponse.class.getName(), true, VerifyRepositoryResponse::new); + private static final ObjectParser PARSER = new ObjectParser<>( + VerifyRepositoryResponse.class.getName(), + true, + VerifyRepositoryResponse::new + ); static { PARSER.declareNamedObjects(VerifyRepositoryResponse::setNodes, NodeView.PARSER, new ParseField("nodes")); } @@ -112,7 +122,7 @@ public VerifyRepositoryResponse(StreamInput in) throws IOException { } public VerifyRepositoryResponse(DiscoveryNode[] nodes) { - this.nodes = Arrays.stream(nodes).map(dn -> new NodeView(dn.getId(), dn.getName())).collect(Collectors.toList()); + this.nodes = Arrays.stream(nodes).map(dn -> new NodeView(dn.getId(), dn.getName())).collect(Collectors.toList()); } public VerifyRepositoryResponse(List nodes) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java index dc5b77f1185e4..2da99fd32d88a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java @@ -15,16 +15,26 @@ import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.Strings; -public class CloneSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder { +public class CloneSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder< + CloneSnapshotRequest, + AcknowledgedResponse, + CloneSnapshotRequestBuilder> { - protected CloneSnapshotRequestBuilder(ElasticsearchClient client, ActionType action, - CloneSnapshotRequest request) { + protected CloneSnapshotRequestBuilder( + ElasticsearchClient client, + ActionType action, + CloneSnapshotRequest request + ) { super(client, action, request); } - public CloneSnapshotRequestBuilder(ElasticsearchClient client, ActionType action, - String repository, String source, String target) { + public CloneSnapshotRequestBuilder( + ElasticsearchClient client, + ActionType action, + String repository, + String source, + String target + ) { this(client, action, new CloneSnapshotRequest(repository, source, target, Strings.EMPTY_ARRAY)); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/TransportCloneSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/TransportCloneSnapshotAction.java index 610f2df53c4d0..1cb8af81d12c3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/TransportCloneSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/clone/TransportCloneSnapshotAction.java @@ -31,11 +31,24 @@ public final class TransportCloneSnapshotAction extends AcknowledgedTransportMas private final SnapshotsService snapshotsService; @Inject - public TransportCloneSnapshotAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, SnapshotsService snapshotsService, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(CloneSnapshotAction.NAME, transportService, clusterService, threadPool, actionFilters, CloneSnapshotRequest::new, - indexNameExpressionResolver, ThreadPool.Names.SAME); + public TransportCloneSnapshotAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + SnapshotsService snapshotsService, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + CloneSnapshotAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + CloneSnapshotRequest::new, + indexNameExpressionResolver, + ThreadPool.Names.SAME + ); this.snapshotsService = snapshotsService; } @@ -46,8 +59,12 @@ protected ClusterBlockException checkBlock(CloneSnapshotRequest request, Cluster } @Override - protected void masterOperation(Task task, final CloneSnapshotRequest request, ClusterState state, - final ActionListener listener) { + protected void masterOperation( + Task task, + final CloneSnapshotRequest request, + ClusterState state, + final ActionListener listener + ) { snapshotsService.cloneSnapshot(request, listener.map(v -> AcknowledgedResponse.TRUE)); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java index 5f2856b05d254..eb3a3dfdaebb9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotAction.java @@ -22,4 +22,3 @@ private CreateSnapshotAction() { super(NAME, CreateSnapshotResponse::new); } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java index 4ea35e6f232fc..47d01c9335c39 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java @@ -50,7 +50,9 @@ * */ public class CreateSnapshotRequest extends MasterNodeRequest - implements IndicesRequest.Replaceable, ToXContentObject { + implements + IndicesRequest.Replaceable, + ToXContentObject { public static final Version SETTINGS_IN_REQUEST_VERSION = Version.V_8_0_0; @@ -74,8 +76,7 @@ public class CreateSnapshotRequest extends MasterNodeRequest userMetadata; - public CreateSnapshotRequest() { - } + public CreateSnapshotRequest() {} /** * Constructs a new put repository request with the provided snapshot and repository names @@ -148,8 +149,10 @@ public ActionRequestValidationException validate() { } final int metadataSize = metadataSize(userMetadata); if (metadataSize > MAXIMUM_METADATA_BYTES) { - validationException = addValidationError("metadata must be smaller than 1024 bytes, but was [" + metadataSize + "]", - validationException); + validationException = addValidationError( + "metadata must be smaller than 1024 bytes, but was [" + metadataSize + "]", + validationException + ); } return validationException; } @@ -442,22 +445,21 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; CreateSnapshotRequest that = (CreateSnapshotRequest) o; - return partial == that.partial && - includeGlobalState == that.includeGlobalState && - waitForCompletion == that.waitForCompletion && - Objects.equals(snapshot, that.snapshot) && - Objects.equals(repository, that.repository) && - Arrays.equals(indices, that.indices) && - Objects.equals(indicesOptions, that.indicesOptions) && - Arrays.equals(featureStates, that.featureStates) && - Objects.equals(masterNodeTimeout, that.masterNodeTimeout) && - Objects.equals(userMetadata, that.userMetadata); + return partial == that.partial + && includeGlobalState == that.includeGlobalState + && waitForCompletion == that.waitForCompletion + && Objects.equals(snapshot, that.snapshot) + && Objects.equals(repository, that.repository) + && Arrays.equals(indices, that.indices) + && Objects.equals(indicesOptions, that.indicesOptions) + && Arrays.equals(featureStates, that.featureStates) + && Objects.equals(masterNodeTimeout, that.masterNodeTimeout) + && Objects.equals(userMetadata, that.userMetadata); } @Override public int hashCode() { - int result = Objects.hash(snapshot, repository, indicesOptions, partial, includeGlobalState, - waitForCompletion, userMetadata); + int result = Objects.hash(snapshot, repository, indicesOptions, partial, includeGlobalState, waitForCompletion, userMetadata); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(featureStates); return result; @@ -465,17 +467,29 @@ public int hashCode() { @Override public String toString() { - return "CreateSnapshotRequest{" + - "snapshot='" + snapshot + '\'' + - ", repository='" + repository + '\'' + - ", indices=" + (indices == null ? null : Arrays.asList(indices)) + - ", indicesOptions=" + indicesOptions + - ", featureStates=" + Arrays.asList(featureStates) + - ", partial=" + partial + - ", includeGlobalState=" + includeGlobalState + - ", waitForCompletion=" + waitForCompletion + - ", masterNodeTimeout=" + masterNodeTimeout + - ", metadata=" + userMetadata + - '}'; + return "CreateSnapshotRequest{" + + "snapshot='" + + snapshot + + '\'' + + ", repository='" + + repository + + '\'' + + ", indices=" + + (indices == null ? null : Arrays.asList(indices)) + + ", indicesOptions=" + + indicesOptions + + ", featureStates=" + + Arrays.asList(featureStates) + + ", partial=" + + partial + + ", includeGlobalState=" + + includeGlobalState + + ", waitForCompletion=" + + waitForCompletion + + ", masterNodeTimeout=" + + masterNodeTimeout + + ", metadata=" + + userMetadata + + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestBuilder.java index 355060834d8f3..33fd896294ecf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestBuilder.java @@ -15,8 +15,10 @@ /** * Create snapshot request builder */ -public class CreateSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder { +public class CreateSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder< + CreateSnapshotRequest, + CreateSnapshotResponse, + CreateSnapshotRequestBuilder> { /** * Constructs a new create snapshot request builder diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponse.java index aed876d01a018..dba03173ab530 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponse.java @@ -29,12 +29,18 @@ */ public class CreateSnapshotResponse extends ActionResponse implements ToXContentObject { - private static final ObjectParser PARSER = - new ObjectParser<>(CreateSnapshotResponse.class.getName(), true, CreateSnapshotResponse::new); + private static final ObjectParser PARSER = new ObjectParser<>( + CreateSnapshotResponse.class.getName(), + true, + CreateSnapshotResponse::new + ); static { - PARSER.declareObject(CreateSnapshotResponse::setSnapshotInfoFromBuilder, - SnapshotInfo.SNAPSHOT_INFO_PARSER, new ParseField("snapshot")); + PARSER.declareObject( + CreateSnapshotResponse::setSnapshotInfoFromBuilder, + SnapshotInfo.SNAPSHOT_INFO_PARSER, + new ParseField("snapshot") + ); } @Nullable @@ -103,9 +109,7 @@ public static CreateSnapshotResponse fromXContent(XContentParser parser) { @Override public String toString() { - return "CreateSnapshotResponse{" + - "snapshotInfo=" + snapshotInfo + - '}'; + return "CreateSnapshotResponse{" + "snapshotInfo=" + snapshotInfo + '}'; } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java index 4cf7d8f913f61..66cf927af7d61 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/create/TransportCreateSnapshotAction.java @@ -29,11 +29,25 @@ public class TransportCreateSnapshotAction extends TransportMasterNodeAction listener) { + protected void masterOperation( + Task task, + final CreateSnapshotRequest request, + ClusterState state, + final ActionListener listener + ) { if (request.waitForCompletion()) { snapshotsService.executeSnapshot(request, listener.map(CreateSnapshotResponse::new)); } else { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java index 31b4b7396241c..9d5e30b604702 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotAction.java @@ -24,4 +24,3 @@ private DeleteSnapshotAction() { } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotRequest.java index 8ea4d25835163..ce410efe79f40 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/DeleteSnapshotRequest.java @@ -32,8 +32,7 @@ public class DeleteSnapshotRequest extends MasterNodeRequest { +public class DeleteSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder< + DeleteSnapshotRequest, + AcknowledgedResponse, + DeleteSnapshotRequestBuilder> { /** * Constructs delete snapshot request builder diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java index 799a7e8f5b45b..32fdb285cb808 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java @@ -30,11 +30,24 @@ public class TransportDeleteSnapshotAction extends AcknowledgedTransportMasterNo private final SnapshotsService snapshotsService; @Inject - public TransportDeleteSnapshotAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, SnapshotsService snapshotsService, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(DeleteSnapshotAction.NAME, transportService, clusterService, threadPool, actionFilters, - DeleteSnapshotRequest::new, indexNameExpressionResolver, ThreadPool.Names.SAME); + public TransportDeleteSnapshotAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + SnapshotsService snapshotsService, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + DeleteSnapshotAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + DeleteSnapshotRequest::new, + indexNameExpressionResolver, + ThreadPool.Names.SAME + ); this.snapshotsService = snapshotsService; } @@ -45,8 +58,12 @@ protected ClusterBlockException checkBlock(DeleteSnapshotRequest request, Cluste } @Override - protected void masterOperation(Task task, final DeleteSnapshotRequest request, ClusterState state, - final ActionListener listener) { + protected void masterOperation( + Task task, + final DeleteSnapshotRequest request, + ClusterState state, + final ActionListener listener + ) { snapshotsService.deleteSnapshots(request, listener.map(v -> AcknowledgedResponse.TRUE)); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateRequest.java index 62a2b7d78c320..e2145f62994ba 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateRequest.java @@ -18,8 +18,7 @@ /** Request for resetting feature state */ public class ResetFeatureStateRequest extends ActionRequest { - public ResetFeatureStateRequest() { - } + public ResetFeatureStateRequest() {} public ResetFeatureStateRequest(StreamInput in) throws IOException { super(in); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateResponse.java index 0c47c38ab726d..45284a0813c4b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/ResetFeatureStateResponse.java @@ -87,9 +87,7 @@ public int hashCode() { @Override public String toString() { - return "ResetFeatureStateResponse{" + - "resetFeatureStateStatusList=" + resetFeatureStateStatusList + - '}'; + return "ResetFeatureStateResponse{" + "resetFeatureStateStatusList=" + resetFeatureStateStatusList + '}'; } /** @@ -126,10 +124,7 @@ public static ResetFeatureStateStatus success(String featureName) { * @return Failure status for a feature */ public static ResetFeatureStateStatus failure(String featureName, Exception exception) { - return new ResetFeatureStateStatus( - featureName, - Status.FAILURE, - exception); + return new ResetFeatureStateStatus(featureName, Status.FAILURE, exception); } private ResetFeatureStateStatus(String featureName, Status status, @Nullable Exception exception) { @@ -218,11 +213,16 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String toString() { - return "ResetFeatureStateStatus{" + - "featureName='" + featureName + '\'' + - ", status=" + status + - ", exception='" + exception + '\'' + - '}'; + return "ResetFeatureStateStatus{" + + "featureName='" + + featureName + + '\'' + + ", status=" + + status + + ", exception='" + + exception + + '\'' + + '}'; } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportResetFeatureStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportResetFeatureStateAction.java index 488c29f339d74..de717942dabf2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportResetFeatureStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportResetFeatureStateAction.java @@ -39,18 +39,14 @@ public TransportResetFeatureStateAction( NodeClient client, ClusterService clusterService ) { - super(ResetFeatureStateAction.NAME, transportService, actionFilters, - ResetFeatureStateRequest::new); + super(ResetFeatureStateAction.NAME, transportService, actionFilters, ResetFeatureStateRequest::new); this.systemIndices = systemIndices; this.client = client; this.clusterService = clusterService; } @Override - protected void doExecute( - Task task, - ResetFeatureStateRequest request, - ActionListener listener) { + protected void doExecute(Task task, ResetFeatureStateRequest request, ActionListener listener) { if (systemIndices.getFeatures().size() == 0) { listener.onResponse(new ResetFeatureStateResponse(Collections.emptyList())); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportSnapshottableFeaturesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportSnapshottableFeaturesAction.java index 62f32bc460f0e..d55b26fe80b13 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportSnapshottableFeaturesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/features/TransportSnapshottableFeaturesAction.java @@ -24,29 +24,56 @@ import java.util.stream.Collectors; -public class TransportSnapshottableFeaturesAction extends TransportMasterNodeAction { private final SystemIndices systemIndices; @Inject - public TransportSnapshottableFeaturesAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, - ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - SystemIndices systemIndices) { - super(SnapshottableFeaturesAction.NAME, transportService, clusterService, threadPool, actionFilters, - GetSnapshottableFeaturesRequest::new, indexNameExpressionResolver, GetSnapshottableFeaturesResponse::new, - ThreadPool.Names.SAME); + public TransportSnapshottableFeaturesAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, + SystemIndices systemIndices + ) { + super( + SnapshottableFeaturesAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + GetSnapshottableFeaturesRequest::new, + indexNameExpressionResolver, + GetSnapshottableFeaturesResponse::new, + ThreadPool.Names.SAME + ); this.systemIndices = systemIndices; } @Override - protected void masterOperation(Task task, GetSnapshottableFeaturesRequest request, ClusterState state, - ActionListener listener) throws Exception { - listener.onResponse(new GetSnapshottableFeaturesResponse(systemIndices.getFeatures().entrySet().stream() - .map(featureEntry -> new GetSnapshottableFeaturesResponse.SnapshottableFeature( - featureEntry.getKey(), - featureEntry.getValue().getDescription())) - .collect(Collectors.toList()))); + protected void masterOperation( + Task task, + GetSnapshottableFeaturesRequest request, + ClusterState state, + ActionListener listener + ) throws Exception { + listener.onResponse( + new GetSnapshottableFeaturesResponse( + systemIndices.getFeatures() + .entrySet() + .stream() + .map( + featureEntry -> new GetSnapshottableFeaturesResponse.SnapshottableFeature( + featureEntry.getKey(), + featureEntry.getValue().getDescription() + ) + ) + .collect(Collectors.toList()) + ) + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java index 7d397f2678c3b..53eab1f6e1d77 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsAction.java @@ -23,4 +23,3 @@ private GetSnapshotsAction() { } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java index 1c60d3e85f3db..12ac39e9b0449 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java @@ -41,8 +41,7 @@ public class GetSnapshotsRequest extends MasterNodeRequest private boolean verbose = DEFAULT_VERBOSE_MODE; - public GetSnapshotsRequest() { - } + public GetSnapshotsRequest() {} /** * Constructs a new get snapshots request with given repository names and list of snapshots @@ -69,7 +68,7 @@ public GetSnapshotsRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { repositories = in.readStringArray(); } else { - repositories = new String[]{in.readString()}; + repositories = new String[] { in.readString() }; } snapshots = in.readStringArray(); ignoreUnavailable = in.readBoolean(); @@ -83,8 +82,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(repositories); } else { if (repositories.length != 1) { - throw new IllegalArgumentException("Requesting snapshots from multiple repositories is not supported in versions prior " + - "to " + MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString()); + throw new IllegalArgumentException( + "Requesting snapshots from multiple repositories is not supported in versions prior " + + "to " + + MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString() + ); } out.writeString(repositories[0]); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java index 260392a901b21..14f9aa46eb6a0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java @@ -15,8 +15,10 @@ /** * Get snapshots request builder */ -public class GetSnapshotsRequestBuilder extends MasterNodeOperationRequestBuilder { +public class GetSnapshotsRequestBuilder extends MasterNodeOperationRequestBuilder< + GetSnapshotsRequest, + GetSnapshotsResponse, + GetSnapshotsRequestBuilder> { /** * Constructs the new get snapshot request with specified repositories @@ -53,7 +55,7 @@ public GetSnapshotsRequestBuilder setSnapshots(String... snapshots) { * @return this builder */ public GetSnapshotsRequestBuilder setCurrentSnapshot() { - request.snapshots(new String[]{GetSnapshotsRequest.CURRENT_SNAPSHOT}); + request.snapshots(new String[] { GetSnapshotsRequest.CURRENT_SNAPSHOT }); return this; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java index 3f1e68cae1e19..457eedd86b9c2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java @@ -34,13 +34,18 @@ */ public class GetSnapshotsResponse extends ActionResponse implements ToXContentObject { - private static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(GetSnapshotsResponse.class.getName(), true, - (args) -> new GetSnapshotsResponse((List) args[0])); + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + GetSnapshotsResponse.class.getName(), + true, + (args) -> new GetSnapshotsResponse((List) args[0]) + ); static { - PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), - (p, c) -> Response.fromXContent(p), new ParseField("responses")); + PARSER.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> Response.fromXContent(p), + new ParseField("responses") + ); } public GetSnapshotsResponse(StreamInput in) throws IOException { @@ -55,23 +60,29 @@ public GetSnapshotsResponse(StreamInput in) throws IOException { } } - public static class Response { private final String repository; private final List snapshots; private final ElasticsearchException error; - private static final ConstructingObjectParser RESPONSE_PARSER = - new ConstructingObjectParser<>(Response.class.getName(), true, - (args) -> new Response((String) args[0], - (List) args[1], (ElasticsearchException) args[2])); + private static final ConstructingObjectParser RESPONSE_PARSER = new ConstructingObjectParser<>( + Response.class.getName(), + true, + (args) -> new Response((String) args[0], (List) args[1], (ElasticsearchException) args[2]) + ); static { RESPONSE_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("repository")); - RESPONSE_PARSER.declareObjectArray(ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> SnapshotInfo.SNAPSHOT_INFO_PARSER.apply(p, c).build(), new ParseField("snapshots")); - RESPONSE_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), - (p, c) -> ElasticsearchException.fromXContent(p), new ParseField("error")); + RESPONSE_PARSER.declareObjectArray( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> SnapshotInfo.SNAPSHOT_INFO_PARSER.apply(p, c).build(), + new ParseField("snapshots") + ); + RESPONSE_PARSER.declareObject( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ElasticsearchException.fromXContent(p), + new ParseField("error") + ); } private Response(String repository, List snapshots, ElasticsearchException error) { @@ -194,8 +205,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(failedResponses, StreamOutput::writeString, StreamOutput::writeException); } else { if (successfulResponses.size() + failedResponses.size() != 1) { - throw new IllegalArgumentException("Requesting snapshots from multiple repositories is not supported in versions prior " + - "to " + GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString()); + throw new IllegalArgumentException( + "Requesting snapshots from multiple repositories is not supported in versions prior " + + "to " + + GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString() + ); } if (successfulResponses.size() == 1) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index ea632f4813315..522decbe6d703 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -66,11 +66,25 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction listener) { + protected void masterOperation( + Task task, + final GetSnapshotsRequest request, + final ClusterState state, + final ActionListener listener + ) { assert task instanceof CancellableTask : task + " not cancellable"; getMultipleReposSnapshotInfo( - state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), - TransportGetRepositoriesAction.getRepositories(state, request.repositories()), - request.snapshots(), - request.ignoreUnavailable(), - request.verbose(), - (CancellableTask) task, - listener + state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), + TransportGetRepositoriesAction.getRepositories(state, request.repositories()), + request.snapshots(), + request.ignoreUnavailable(), + request.verbose(), + (CancellableTask) task, + listener ); } - private void getMultipleReposSnapshotInfo(SnapshotsInProgress snapshotsInProgress, - List repos, - String[] snapshots, - boolean ignoreUnavailable, - boolean verbose, - CancellableTask cancellableTask, - ActionListener listener) { + private void getMultipleReposSnapshotInfo( + SnapshotsInProgress snapshotsInProgress, + List repos, + String[] snapshots, + boolean ignoreUnavailable, + boolean verbose, + CancellableTask cancellableTask, + ActionListener listener + ) { // short-circuit if there are no repos, because we can not create GroupedActionListener of size 0 if (repos.isEmpty()) { listener.onResponse(new GetSnapshotsResponse(Collections.emptyList())); return; } - final GroupedActionListener groupedActionListener = - new GroupedActionListener<>( - listener.map(responses -> { - assert repos.size() == responses.size(); - return new GetSnapshotsResponse(responses); - }), repos.size()); + final GroupedActionListener groupedActionListener = new GroupedActionListener<>( + listener.map(responses -> { + assert repos.size() == responses.size(); + return new GetSnapshotsResponse(responses); + }), + repos.size() + ); for (final RepositoryMetadata repo : repos) { final String repoName = repo.name(); - getSingleRepoSnapshotInfo(snapshotsInProgress, repoName, snapshots, ignoreUnavailable, verbose, cancellableTask, - groupedActionListener.delegateResponse((groupedListener, e) -> { - if (e instanceof ElasticsearchException) { - groupedListener.onResponse(GetSnapshotsResponse.Response.error(repoName, (ElasticsearchException) e)); - } else { - groupedListener.onFailure(e); - } - }).map(snInfos -> GetSnapshotsResponse.Response.snapshots(repoName, snInfos))); + getSingleRepoSnapshotInfo( + snapshotsInProgress, + repoName, + snapshots, + ignoreUnavailable, + verbose, + cancellableTask, + groupedActionListener.delegateResponse((groupedListener, e) -> { + if (e instanceof ElasticsearchException) { + groupedListener.onResponse(GetSnapshotsResponse.Response.error(repoName, (ElasticsearchException) e)); + } else { + groupedListener.onFailure(e); + } + }).map(snInfos -> GetSnapshotsResponse.Response.snapshots(repoName, snInfos)) + ); } } - private void getSingleRepoSnapshotInfo(SnapshotsInProgress snapshotsInProgress, String repo, String[] snapshots, - boolean ignoreUnavailable, boolean verbose, CancellableTask task, - ActionListener> listener) { + private void getSingleRepoSnapshotInfo( + SnapshotsInProgress snapshotsInProgress, + String repo, + String[] snapshots, + boolean ignoreUnavailable, + boolean verbose, + CancellableTask task, + ActionListener> listener + ) { final Map allSnapshotIds = new HashMap<>(); final List currentSnapshots = new ArrayList<>(); for (SnapshotInfo snapshotInfo : sortedCurrentSnapshots(snapshotsInProgress, repo)) { @@ -145,8 +179,21 @@ private void getSingleRepoSnapshotInfo(SnapshotsInProgress snapshotsInProgress, repositoriesService.getRepositoryData(repo, repositoryDataListener); } - repositoryDataListener.whenComplete(repositoryData -> loadSnapshotInfos(snapshotsInProgress, repo, snapshots, - ignoreUnavailable, verbose, allSnapshotIds, currentSnapshots, repositoryData, task, listener), listener::onFailure); + repositoryDataListener.whenComplete( + repositoryData -> loadSnapshotInfos( + snapshotsInProgress, + repo, + snapshots, + ignoreUnavailable, + verbose, + allSnapshotIds, + currentSnapshots, + repositoryData, + task, + listener + ), + listener::onFailure + ); } /** @@ -158,8 +205,11 @@ private void getSingleRepoSnapshotInfo(SnapshotsInProgress snapshotsInProgress, */ private static List sortedCurrentSnapshots(SnapshotsInProgress snapshotsInProgress, String repositoryName) { List snapshotList = new ArrayList<>(); - List entries = - SnapshotsService.currentSnapshots(snapshotsInProgress, repositoryName, Collections.emptyList()); + List entries = SnapshotsService.currentSnapshots( + snapshotsInProgress, + repositoryName, + Collections.emptyList() + ); for (SnapshotsInProgress.Entry entry : entries) { snapshotList.add(new SnapshotInfo(entry)); } @@ -167,11 +217,18 @@ private static List sortedCurrentSnapshots(SnapshotsInProgress sna return unmodifiableList(snapshotList); } - - private void loadSnapshotInfos(SnapshotsInProgress snapshotsInProgress, String repo, String[] snapshots, - boolean ignoreUnavailable, boolean verbose, Map allSnapshotIds, - List currentSnapshots, @Nullable RepositoryData repositoryData, - CancellableTask task, ActionListener> listener) { + private void loadSnapshotInfos( + SnapshotsInProgress snapshotsInProgress, + String repo, + String[] snapshots, + boolean ignoreUnavailable, + boolean verbose, + Map allSnapshotIds, + List currentSnapshots, + @Nullable RepositoryData repositoryData, + CancellableTask task, + ActionListener> listener + ) { if (task.isCancelled()) { listener.onFailure(new TaskCancelledException("task cancelled")); return; @@ -235,12 +292,14 @@ private void loadSnapshotInfos(SnapshotsInProgress snapshotsInProgress, String r * @param ignoreUnavailable if true, snapshots that could not be read will only be logged with a warning, * if false, they will throw an error */ - private void snapshots(SnapshotsInProgress snapshotsInProgress, - String repositoryName, - Collection snapshotIds, - boolean ignoreUnavailable, - CancellableTask task, - ActionListener> listener) { + private void snapshots( + SnapshotsInProgress snapshotsInProgress, + String repositoryName, + Collection snapshotIds, + boolean ignoreUnavailable, + CancellableTask task, + ActionListener> listener + ) { if (task.isCancelled()) { listener.onFailure(new TaskCancelledException("task cancelled")); return; @@ -249,7 +308,10 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, final Set snapshotIdsToIterate = new HashSet<>(snapshotIds); // first, look at the snapshots in progress final List entries = SnapshotsService.currentSnapshots( - snapshotsInProgress, repositoryName, snapshotIdsToIterate.stream().map(SnapshotId::getName).collect(Collectors.toList())); + snapshotsInProgress, + repositoryName, + snapshotIdsToIterate.stream().map(SnapshotId::getName).collect(Collectors.toList()) + ); for (SnapshotsInProgress.Entry entry : entries) { if (snapshotIdsToIterate.remove(entry.snapshot().getSnapshotId())) { snapshotSet.add(new SnapshotInfo(entry)); @@ -279,7 +341,8 @@ private void snapshots(SnapshotsInProgress snapshotsInProgress, listener.onFailure(e); return; } - repository.getSnapshotInfo(new GetSnapshotInfoContext( + repository.getSnapshotInfo( + new GetSnapshotInfoContext( snapshotIdsToIterate, ignoreUnavailable == false, task::isCancelled, @@ -308,9 +371,11 @@ private boolean isCurrentSnapshotsOnly(String[] snapshots) { return (snapshots.length == 1 && GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshots[0])); } - private static List buildSimpleSnapshotInfos(final Set toResolve, - final RepositoryData repositoryData, - final List currentSnapshots) { + private static List buildSimpleSnapshotInfos( + final Set toResolve, + final RepositoryData repositoryData, + final List currentSnapshots + ) { List snapshotInfos = new ArrayList<>(); for (SnapshotInfo snapshotInfo : currentSnapshots) { if (toResolve.remove(snapshotInfo.snapshotId())) { @@ -321,16 +386,22 @@ private static List buildSimpleSnapshotInfos(final Set for (IndexId indexId : repositoryData.getIndices().values()) { for (SnapshotId snapshotId : repositoryData.getSnapshots(indexId)) { if (toResolve.contains(snapshotId)) { - snapshotsToIndices.computeIfAbsent(snapshotId, (k) -> new ArrayList<>()) - .add(indexId.getName()); + snapshotsToIndices.computeIfAbsent(snapshotId, (k) -> new ArrayList<>()).add(indexId.getName()); } } } for (SnapshotId snapshotId : toResolve) { final List indices = snapshotsToIndices.getOrDefault(snapshotId, Collections.emptyList()); CollectionUtil.timSort(indices); - snapshotInfos.add(new SnapshotInfo(snapshotId, indices, Collections.emptyList(), Collections.emptyList(), - repositoryData.getSnapshotState(snapshotId))); + snapshotInfos.add( + new SnapshotInfo( + snapshotId, + indices, + Collections.emptyList(), + Collections.emptyList(), + repositoryData.getSnapshotState(snapshotId) + ) + ); } CollectionUtil.timSort(snapshotInfos); return Collections.unmodifiableList(snapshotInfos); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java index 1c29d19b3aa99..2f3e92d2f55a9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreClusterStateListener.java @@ -30,9 +30,11 @@ public class RestoreClusterStateListener implements ClusterStateListener { private final String uuid; private final ActionListener listener; - - private RestoreClusterStateListener(ClusterService clusterService, RestoreService.RestoreCompletionResponse response, - ActionListener listener) { + private RestoreClusterStateListener( + ClusterService clusterService, + RestoreService.RestoreCompletionResponse response, + ActionListener listener + ) { this.clusterService = clusterService; this.uuid = response.getUuid(); this.listener = listener; @@ -53,10 +55,12 @@ public void clusterChanged(ClusterChangedEvent changedEvent) { ImmutableOpenMap shards = prevEntry.shards(); assert prevEntry.state().completed() : "expected completed snapshot state but was " + prevEntry.state(); assert RestoreService.completed(shards) : "expected all restore entries to be completed"; - RestoreInfo ri = new RestoreInfo(prevEntry.snapshot().getSnapshotId().getName(), + RestoreInfo ri = new RestoreInfo( + prevEntry.snapshot().getSnapshotId().getName(), prevEntry.indices(), shards.size(), - shards.size() - RestoreService.failedShards(shards)); + shards.size() - RestoreService.failedShards(shards) + ); RestoreSnapshotResponse response = new RestoreSnapshotResponse(ri); logger.debug("restore of [{}] completed", prevEntry.snapshot().getSnapshotId()); listener.onResponse(response); @@ -69,8 +73,11 @@ public void clusterChanged(ClusterChangedEvent changedEvent) { * Creates a cluster state listener and registers it with the cluster service. The listener passed as a * parameter will be called when the restore is complete. */ - public static void createAndRegisterListener(ClusterService clusterService, RestoreService.RestoreCompletionResponse response, - ActionListener listener) { + public static void createAndRegisterListener( + ClusterService clusterService, + RestoreService.RestoreCompletionResponse response, + ActionListener listener + ) { clusterService.addListener(new RestoreClusterStateListener(clusterService, response, listener)); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java index 39e56f3e0cd50..a3b6db6ad97c7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotAction.java @@ -22,4 +22,3 @@ private RestoreSnapshotAction() { super(NAME, RestoreSnapshotResponse::new); } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java index 1d2131a8819c7..fe2ef130e479a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java @@ -58,8 +58,7 @@ public class RestoreSnapshotRequest extends MasterNodeRequest source) { } indexSettings((Map) entry.getValue()); } else if (name.equals("ignore_index_settings")) { - if (entry.getValue() instanceof String) { - ignoreIndexSettings(Strings.splitStringByCommaToArray((String) entry.getValue())); - } else if (entry.getValue() instanceof List) { - ignoreIndexSettings((List) entry.getValue()); - } else { - throw new IllegalArgumentException("malformed ignore_index_settings section, should be an array of strings"); - } + if (entry.getValue() instanceof String) { + ignoreIndexSettings(Strings.splitStringByCommaToArray((String) entry.getValue())); + } else if (entry.getValue() instanceof List) { + ignoreIndexSettings((List) entry.getValue()); + } else { + throw new IllegalArgumentException("malformed ignore_index_settings section, should be an array of strings"); + } } else { if (IndicesOptions.isIndicesOptions(name) == false) { throw new IllegalArgumentException("Unknown parameter " + name); @@ -599,27 +598,39 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; RestoreSnapshotRequest that = (RestoreSnapshotRequest) o; - return waitForCompletion == that.waitForCompletion && - includeGlobalState == that.includeGlobalState && - partial == that.partial && - includeAliases == that.includeAliases && - Objects.equals(snapshot, that.snapshot) && - Objects.equals(repository, that.repository) && - Arrays.equals(indices, that.indices) && - Objects.equals(indicesOptions, that.indicesOptions) && - Arrays.equals(featureStates, that.featureStates) && - Objects.equals(renamePattern, that.renamePattern) && - Objects.equals(renameReplacement, that.renameReplacement) && - Objects.equals(indexSettings, that.indexSettings) && - Arrays.equals(ignoreIndexSettings, that.ignoreIndexSettings) && - Objects.equals(snapshotUuid, that.snapshotUuid) && - skipOperatorOnlyState == that.skipOperatorOnlyState; + return waitForCompletion == that.waitForCompletion + && includeGlobalState == that.includeGlobalState + && partial == that.partial + && includeAliases == that.includeAliases + && Objects.equals(snapshot, that.snapshot) + && Objects.equals(repository, that.repository) + && Arrays.equals(indices, that.indices) + && Objects.equals(indicesOptions, that.indicesOptions) + && Arrays.equals(featureStates, that.featureStates) + && Objects.equals(renamePattern, that.renamePattern) + && Objects.equals(renameReplacement, that.renameReplacement) + && Objects.equals(indexSettings, that.indexSettings) + && Arrays.equals(ignoreIndexSettings, that.ignoreIndexSettings) + && Objects.equals(snapshotUuid, that.snapshotUuid) + && skipOperatorOnlyState == that.skipOperatorOnlyState; } @Override public int hashCode() { - int result = Objects.hash(snapshot, repository, indicesOptions, renamePattern, renameReplacement, waitForCompletion, - includeGlobalState, partial, includeAliases, indexSettings, snapshotUuid, skipOperatorOnlyState); + int result = Objects.hash( + snapshot, + repository, + indicesOptions, + renamePattern, + renameReplacement, + waitForCompletion, + includeGlobalState, + partial, + includeAliases, + indexSettings, + snapshotUuid, + skipOperatorOnlyState + ); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(ignoreIndexSettings); result = 31 * result + Arrays.hashCode(featureStates); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java index ceab46e73fc63..1476822377db3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java @@ -20,8 +20,10 @@ /** * Restore snapshot request builder */ -public class RestoreSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder { +public class RestoreSnapshotRequestBuilder extends MasterNodeOperationRequestBuilder< + RestoreSnapshotRequest, + RestoreSnapshotResponse, + RestoreSnapshotRequestBuilder> { /** * Constructs new restore snapshot request builder @@ -37,7 +39,6 @@ public RestoreSnapshotRequestBuilder(ElasticsearchClient client, RestoreSnapshot super(client, action, new RestoreSnapshotRequest(repository, name)); } - /** * Sets snapshot name * @@ -87,7 +88,6 @@ public RestoreSnapshotRequestBuilder setIndicesOptions(IndicesOptions indicesOpt return this; } - /** * Sets rename pattern that should be applied to restored indices. *

@@ -207,7 +207,6 @@ public RestoreSnapshotRequestBuilder setIndexSettings(Map source return this; } - /** * Sets the list of index settings and index settings groups that shouldn't be restored from snapshot */ diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java index 503a3594a2f88..156ea9aa7ffb3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java @@ -78,21 +78,22 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par } public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "restore_snapshot", true, v -> { + "restore_snapshot", + true, + v -> { RestoreInfo restoreInfo = (RestoreInfo) v[0]; Boolean accepted = (Boolean) v[1]; - assert (accepted == null && restoreInfo != null) || - (accepted != null && accepted && restoreInfo == null) : - "accepted: [" + accepted + "], restoreInfo: [" + restoreInfo + "]"; + assert (accepted == null && restoreInfo != null) || (accepted != null && accepted && restoreInfo == null) + : "accepted: [" + accepted + "], restoreInfo: [" + restoreInfo + "]"; return new RestoreSnapshotResponse(restoreInfo); - }); + } + ); static { PARSER.declareObject(optionalConstructorArg(), (parser, context) -> RestoreInfo.fromXContent(parser), new ParseField("snapshot")); PARSER.declareBoolean(optionalConstructorArg(), new ParseField("accepted")); } - public static RestoreSnapshotResponse fromXContent(XContentParser parser) throws IOException { return PARSER.parse(parser, null); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java index 5f7e89e2855d2..7b247f1b14a42 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/TransportRestoreSnapshotAction.java @@ -29,11 +29,25 @@ public class TransportRestoreSnapshotAction extends TransportMasterNodeAction listener) { + protected void masterOperation( + Task task, + final RestoreSnapshotRequest request, + final ClusterState state, + final ActionListener listener + ) { restoreService.restoreSnapshot(request, listener.delegateFailure((delegatedListener, restoreCompletionResponse) -> { if (restoreCompletionResponse.getRestoreInfo() == null && request.waitForCompletion()) { RestoreClusterStateListener.createAndRegisterListener(clusterService, restoreCompletionResponse, delegatedListener); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStage.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStage.java index d606ebf588357..14811638a2cb9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStage.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStage.java @@ -8,29 +8,28 @@ package org.elasticsearch.action.admin.cluster.snapshots.status; - public enum SnapshotIndexShardStage { /** * Snapshot hasn't started yet */ - INIT((byte)0, false), + INIT((byte) 0, false), /** * Index files are being copied */ - STARTED((byte)1, false), + STARTED((byte) 1, false), /** * Snapshot metadata is being written or this shard's status in the cluster state is being updated */ - FINALIZE((byte)2, false), + FINALIZE((byte) 2, false), /** * Snapshot completed successfully */ - DONE((byte)3, true), + DONE((byte) 3, true), /** * Snapshot failed */ - FAILURE((byte)4, true); + FAILURE((byte) 4, true); private byte value; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStatus.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStatus.java index 334f1aab6bc45..79e9cd7a758de 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStatus.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexShardStatus.java @@ -79,9 +79,16 @@ public SnapshotIndexShardStatus(StreamInput in) throws IOException { default: throw new IllegalArgumentException("Unknown stage type " + indexShardStatus.getStage()); } - this.stats = new SnapshotStats(indexShardStatus.getStartTime(), indexShardStatus.getTotalTime(), - indexShardStatus.getIncrementalFileCount(), indexShardStatus.getTotalFileCount(), indexShardStatus.getProcessedFileCount(), - indexShardStatus.getIncrementalSize(), indexShardStatus.getTotalSize(), indexShardStatus.getProcessedSize()); + this.stats = new SnapshotStats( + indexShardStatus.getStartTime(), + indexShardStatus.getTotalTime(), + indexShardStatus.getIncrementalFileCount(), + indexShardStatus.getTotalFileCount(), + indexShardStatus.getProcessedFileCount(), + indexShardStatus.getIncrementalSize(), + indexShardStatus.getTotalSize(), + indexShardStatus.getProcessedSize() + ); this.failure = indexShardStatus.getFailure(); this.nodeId = nodeId; } @@ -155,7 +162,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws static final ObjectParser.NamedObjectParser PARSER; static { ConstructingObjectParser innerParser = new ConstructingObjectParser<>( - "snapshot_index_shard_status", true, + "snapshot_index_shard_status", + true, (Object[] parsedObjects, ShardId shard) -> { int i = 0; String rawStage = (String) parsedObjects[i++]; @@ -169,7 +177,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } catch (IllegalArgumentException iae) { throw new ElasticsearchParseException( "failed to parse snapshot index shard status [{}][{}], unknown stage [{}]", - shard.getIndex().getName(), shard.getId(), rawStage); + shard.getIndex().getName(), + shard.getId(), + rawStage + ); } return new SnapshotIndexShardStatus(shard, stage, stats, nodeId, failure); } @@ -186,7 +197,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws shard = Integer.parseInt(shardName); } catch (NumberFormatException nfe) { throw new ElasticsearchParseException( - "failed to parse snapshot index shard status [{}], expected numeric shard id but got [{}]", indexId, shardName); + "failed to parse snapshot index shard status [{}], expected numeric shard id but got [{}]", + indexId, + shardName + ); } ShardId shardId = new ShardId(new Index(indexId, IndexMetadata.INDEX_UUID_NA_VALUE), shard); return innerParser.parse(p, shardId); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java index 408ad531c460c..fad72a1ae4aaf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java @@ -54,8 +54,12 @@ public class SnapshotIndexStatus implements Iterable, this.indexShards = unmodifiableMap(indexShards); } - public SnapshotIndexStatus(String index, Map indexShards, SnapshotShardsStats shardsStats, - SnapshotStats stats) { + public SnapshotIndexStatus( + String index, + Map indexShards, + SnapshotShardsStats shardsStats, + SnapshotStats stats + ) { this.index = index; this.indexShards = indexShards; this.shardsStats = shardsStats; @@ -116,13 +120,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws static final ObjectParser.NamedObjectParser PARSER; static { ConstructingObjectParser innerParser = new ConstructingObjectParser<>( - "snapshot_index_status", true, + "snapshot_index_status", + true, (Object[] parsedObjects, String index) -> { int i = 0; SnapshotShardsStats shardsStats = ((SnapshotShardsStats) parsedObjects[i++]); SnapshotStats stats = ((SnapshotStats) parsedObjects[i++]); - @SuppressWarnings("unchecked") List shardStatuses = - (List) parsedObjects[i]; + @SuppressWarnings("unchecked") + List shardStatuses = (List) parsedObjects[i]; final Map indexShards; if (shardStatuses == null || shardStatuses.isEmpty()) { @@ -134,11 +139,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } } return new SnapshotIndexStatus(index, indexShards, shardsStats, stats); - }); - innerParser.declareObject(constructorArg(), (p, c) -> SnapshotShardsStats.PARSER.apply(p, null), - new ParseField(SnapshotShardsStats.Fields.SHARDS_STATS)); - innerParser.declareObject(constructorArg(), (p, c) -> SnapshotStats.fromXContent(p), - new ParseField(SnapshotStats.Fields.STATS)); + } + ); + innerParser.declareObject( + constructorArg(), + (p, c) -> SnapshotShardsStats.PARSER.apply(p, null), + new ParseField(SnapshotShardsStats.Fields.SHARDS_STATS) + ); + innerParser.declareObject(constructorArg(), (p, c) -> SnapshotStats.fromXContent(p), new ParseField(SnapshotStats.Fields.STATS)); innerParser.declareNamedObjects(constructorArg(), SnapshotIndexShardStatus.PARSER, new ParseField(Fields.SHARDS)); PARSER = ((p, c, name) -> innerParser.apply(p, name)); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotShardsStats.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotShardsStats.java index 93ad3daa12990..0a488b7d636c3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotShardsStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotShardsStats.java @@ -57,8 +57,14 @@ public class SnapshotShardsStats implements ToXContentObject { } } - public SnapshotShardsStats(int initializingShards, int startedShards, int finalizingShards, int doneShards, int failedShards, - int totalShards) { + public SnapshotShardsStats( + int initializingShards, + int startedShards, + int finalizingShards, + int doneShards, + int failedShards, + int totalShards + ) { this.initializingShards = initializingShards; this.startedShards = startedShards; this.finalizingShards = finalizingShards; @@ -135,7 +141,8 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par } static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - Fields.SHARDS_STATS, true, + Fields.SHARDS_STATS, + true, (Object[] parsedObjects) -> { int i = 0; int initializingShards = (int) parsedObjects[i++]; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java index 73158f241a649..dca1eec0c26bf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java @@ -49,9 +49,16 @@ public class SnapshotStats implements Writeable, ToXContentObject { totalSize = in.readVLong(); } - SnapshotStats(long startTime, long time, - int incrementalFileCount, int totalFileCount, int processedFileCount, - long incrementalSize, long totalSize, long processedSize) { + SnapshotStats( + long startTime, + long time, + int incrementalFileCount, + int totalFileCount, + int processedFileCount, + long incrementalSize, + long totalSize, + long processedSize + ) { this.startTime = startTime; this.time = time; assert time >= 0 : "Tried to initialize snapshot stats with negative total time [" + time + "]"; @@ -274,8 +281,16 @@ public static SnapshotStats fromXContent(XContentParser parser) throws IOExcepti } } } - return new SnapshotStats(startTime, time, incrementalFileCount, totalFileCount, processedFileCount, incrementalSize, totalSize, - processedSize); + return new SnapshotStats( + startTime, + time, + incrementalFileCount, + totalFileCount, + processedFileCount, + incrementalSize, + totalSize, + processedSize + ); } /** diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java index 555c814186253..60191f7bfb18b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java @@ -70,8 +70,14 @@ public class SnapshotStatus implements ToXContentObject, Writeable { updateShardStats(startTime, time); } - SnapshotStatus(Snapshot snapshot, State state, List shards, Boolean includeGlobalState, - long startTime, long time) { + SnapshotStatus( + Snapshot snapshot, + State state, + List shards, + Boolean includeGlobalState, + long startTime, + long time + ) { this.snapshot = Objects.requireNonNull(snapshot); this.state = Objects.requireNonNull(state); this.shards = Objects.requireNonNull(shards); @@ -81,9 +87,15 @@ public class SnapshotStatus implements ToXContentObject, Writeable { updateShardStats(startTime, time); } - private SnapshotStatus(Snapshot snapshot, State state, List shards, - Map indicesStatus, SnapshotShardsStats shardsStats, - SnapshotStats stats, Boolean includeGlobalState) { + private SnapshotStatus( + Snapshot snapshot, + State state, + List shards, + Map indicesStatus, + SnapshotShardsStats shardsStats, + SnapshotStats stats, + Boolean includeGlobalState + ) { this.snapshot = snapshot; this.state = state; this.shards = shards; @@ -206,7 +218,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "snapshot_status", true, + "snapshot_status", + true, (Object[] parsedObjects) -> { int i = 0; String name = (String) parsedObjects[i++]; @@ -216,7 +229,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws Boolean includeGlobalState = (Boolean) parsedObjects[i++]; SnapshotStats stats = ((SnapshotStats) parsedObjects[i++]); SnapshotShardsStats shardsStats = ((SnapshotShardsStats) parsedObjects[i++]); - @SuppressWarnings("unchecked") List indices = ((List) parsedObjects[i]); + @SuppressWarnings("unchecked") + List indices = ((List) parsedObjects[i]); Snapshot snapshot = new Snapshot(repository, new SnapshotId(name, uuid)); SnapshotsInProgress.State state = SnapshotsInProgress.State.valueOf(rawState); @@ -234,15 +248,20 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } } return new SnapshotStatus(snapshot, state, shards, indicesStatus, shardsStats, stats, includeGlobalState); - }); + } + ); static { PARSER.declareString(constructorArg(), new ParseField(SNAPSHOT)); PARSER.declareString(constructorArg(), new ParseField(REPOSITORY)); PARSER.declareString(constructorArg(), new ParseField(UUID)); PARSER.declareString(constructorArg(), new ParseField(STATE)); PARSER.declareBoolean(optionalConstructorArg(), new ParseField(INCLUDE_GLOBAL_STATE)); - PARSER.declareField(constructorArg(), SnapshotStats::fromXContent, new ParseField(SnapshotStats.Fields.STATS), - ObjectParser.ValueType.OBJECT); + PARSER.declareField( + constructorArg(), + SnapshotStats::fromXContent, + new ParseField(SnapshotStats.Fields.STATS), + ObjectParser.ValueType.OBJECT + ); PARSER.declareObject(constructorArg(), SnapshotShardsStats.PARSER, new ParseField(SnapshotShardsStats.Fields.SHARDS_STATS)); PARSER.declareNamedObjects(constructorArg(), SnapshotIndexStatus.PARSER, new ParseField(INDICES)); } @@ -266,9 +285,12 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; SnapshotStatus that = (SnapshotStatus) o; - return Objects.equals(snapshot, that.snapshot) && state == that.state && Objects.equals(indicesStatus, that.indicesStatus) - && Objects.equals(shardsStats, that.shardsStats) && Objects.equals(stats, that.stats) - && Objects.equals(includeGlobalState, that.includeGlobalState); + return Objects.equals(snapshot, that.snapshot) + && state == that.state + && Objects.equals(indicesStatus, that.indicesStatus) + && Objects.equals(shardsStats, that.shardsStats) + && Objects.equals(stats, that.stats) + && Objects.equals(includeGlobalState, that.includeGlobalState); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java index 78281aafc67af..9f68ac17a25e8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusAction.java @@ -22,4 +22,3 @@ private SnapshotsStatusAction() { super(NAME, SnapshotsStatusResponse::new); } } - diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java index b6fc25e7e2655..fc8cd0bdab407 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java @@ -33,8 +33,7 @@ public class SnapshotsStatusRequest extends MasterNodeRequest { +public class SnapshotsStatusRequestBuilder extends MasterNodeOperationRequestBuilder< + SnapshotsStatusRequest, + SnapshotsStatusResponse, + SnapshotsStatusRequestBuilder> { /** * Constructs the new snapshot status request diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusResponse.java index b1171fa341cdd..0d538ce70d748 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotsStatusResponse.java @@ -67,9 +67,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "snapshots_status_response", true, + "snapshots_status_response", + true, (Object[] parsedObjects) -> { - @SuppressWarnings("unchecked") List snapshots = (List) parsedObjects[0]; + @SuppressWarnings("unchecked") + List snapshots = (List) parsedObjects[0]; return new SnapshotsStatusResponse(snapshots); } ); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java index 0acecc01a9c3f..f55c23647baed 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportNodesSnapshotsStatus.java @@ -42,10 +42,11 @@ /** * Transport action that collects snapshot shard statuses from data nodes */ -public class TransportNodesSnapshotsStatus extends TransportNodesAction { +public class TransportNodesSnapshotsStatus extends TransportNodesAction< + TransportNodesSnapshotsStatus.Request, + TransportNodesSnapshotsStatus.NodesSnapshotStatus, + TransportNodesSnapshotsStatus.NodeRequest, + TransportNodesSnapshotsStatus.NodeSnapshotStatus> { public static final String ACTION_NAME = SnapshotsStatusAction.NAME + "[nodes]"; public static final ActionType TYPE = new ActionType<>(ACTION_NAME, NodesSnapshotStatus::new); @@ -53,11 +54,24 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction nodes) th } } - public static class NodeRequest extends TransportRequest { private final List snapshots; @@ -183,7 +196,8 @@ public static class NodeSnapshotStatus extends BaseNodeResponse { public NodeSnapshotStatus(StreamInput in) throws IOException { super(in); status = unmodifiableMap( - in.readMap(Snapshot::new, input -> unmodifiableMap(input.readMap(ShardId::new, SnapshotIndexShardStatus::new)))); + in.readMap(Snapshot::new, input -> unmodifiableMap(input.readMap(ShardId::new, SnapshotIndexShardStatus::new))) + ); } public NodeSnapshotStatus(DiscoveryNode node, Map> status) { @@ -199,8 +213,11 @@ public Map> status() { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); if (status != null) { - out.writeMap(status, (o, s) -> s.writeTo(o), - (output, v) -> output.writeMap(v, (o, shardId) -> shardId.writeTo(o), (o, sis) -> sis.writeTo(o))); + out.writeMap( + status, + (o, s) -> s.writeTo(o), + (output, v) -> output.writeMap(v, (o, shardId) -> shardId.writeTo(o), (o, sis) -> sis.writeTo(o)) + ); } else { out.writeVInt(0); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 508366bd9246c..203d938dacf3c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -74,11 +74,26 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction listener) throws Exception { + protected void masterOperation( + Task task, + final SnapshotsStatusRequest request, + final ClusterState state, + final ActionListener listener + ) throws Exception { assert task instanceof CancellableTask : task + " not cancellable"; final CancellableTask cancellableTask = (CancellableTask) task; final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - List currentSnapshots = - SnapshotsService.currentSnapshots(snapshotsInProgress, request.repository(), Arrays.asList(request.snapshots())); + List currentSnapshots = SnapshotsService.currentSnapshots( + snapshotsInProgress, + request.repository(), + Arrays.asList(request.snapshots()) + ); if (currentSnapshots.isEmpty()) { buildResponse(snapshotsInProgress, request, currentSnapshots, null, cancellableTask, listener); return; @@ -118,12 +139,19 @@ protected void masterOperation(Task task, final SnapshotsStatusRequest request, for (int i = 0; i < currentSnapshots.size(); i++) { snapshots[i] = currentSnapshots.get(i).snapshot(); } - client.executeLocally(TransportNodesSnapshotsStatus.TYPE, - new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)) - .snapshots(snapshots).timeout(request.masterNodeTimeout()), + client.executeLocally( + TransportNodesSnapshotsStatus.TYPE, + new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)).snapshots(snapshots) + .timeout(request.masterNodeTimeout()), ActionListener.wrap( - nodeSnapshotStatuses -> buildResponse(snapshotsInProgress, request, currentSnapshots, nodeSnapshotStatuses, - cancellableTask, listener), + nodeSnapshotStatuses -> buildResponse( + snapshotsInProgress, + request, + currentSnapshots, + nodeSnapshotStatuses, + cancellableTask, + listener + ), listener::onFailure ) ); @@ -134,12 +162,14 @@ protected void masterOperation(Task task, final SnapshotsStatusRequest request, } - private void buildResponse(SnapshotsInProgress snapshotsInProgress, - SnapshotsStatusRequest request, - List currentSnapshotEntries, - TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses, - CancellableTask task, - ActionListener listener) { + private void buildResponse( + SnapshotsInProgress snapshotsInProgress, + SnapshotsStatusRequest request, + List currentSnapshotEntries, + TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses, + CancellableTask task, + ActionListener listener + ) { // First process snapshot that are currently processed List builder = new ArrayList<>(); Set currentSnapshotNames = new HashSet<>(); @@ -165,15 +195,19 @@ private void buildResponse(SnapshotsInProgress snapshotsInProgress, SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.key); if (shardStatus != null) { // We have full information about this shard - if (shardStatus.getStage() == SnapshotIndexShardStage.DONE - && shardEntry.value.state() != SUCCESS) { + if (shardStatus.getStage() == SnapshotIndexShardStage.DONE && shardEntry.value.state() != SUCCESS) { // Unlikely edge case: // Data node has finished snapshotting the shard but the cluster state has not yet been updated // to reflect this. We adjust the status to show up as snapshot metadata being written because // technically if the data node failed before successfully reporting DONE state to master, then // this shards state would jump to a failed state. - shardStatus = new SnapshotIndexShardStatus(shardEntry.key, SnapshotIndexShardStage.FINALIZE, - shardStatus.getStats(), shardStatus.getNodeId(), shardStatus.getFailure()); + shardStatus = new SnapshotIndexShardStatus( + shardEntry.key, + SnapshotIndexShardStage.FINALIZE, + shardStatus.getStats(), + shardStatus.getNodeId(), + shardStatus.getFailure() + ); } shardStatusBuilder.add(shardStatus); continue; @@ -208,17 +242,31 @@ private void buildResponse(SnapshotsInProgress snapshotsInProgress, // Shard snapshot completed successfully so we should be able to load the exact statistics for this // shard from the repository already. final ShardId shardId = shardEntry.key; - shardStatus = new SnapshotIndexShardStatus(shardId, repositoriesService.repository(entry.repository()) - .getShardSnapshotStatus(entry.snapshot().getSnapshotId(), entry.indices().get(shardId.getIndexName()), - shardId).asCopy()); + shardStatus = new SnapshotIndexShardStatus( + shardId, + repositoriesService.repository(entry.repository()) + .getShardSnapshotStatus( + entry.snapshot().getSnapshotId(), + entry.indices().get(shardId.getIndexName()), + shardId + ) + .asCopy() + ); } else { shardStatus = new SnapshotIndexShardStatus(shardEntry.key, stage); } shardStatusBuilder.add(shardStatus); } - builder.add(new SnapshotStatus(entry.snapshot(), entry.state(), - Collections.unmodifiableList(shardStatusBuilder), entry.includeGlobalState(), entry.startTime(), - Math.max(threadPool.absoluteTimeInMillis() - entry.startTime(), 0L))); + builder.add( + new SnapshotStatus( + entry.snapshot(), + entry.state(), + Collections.unmodifiableList(shardStatusBuilder), + entry.includeGlobalState(), + entry.startTime(), + Math.max(threadPool.absoluteTimeInMillis() - entry.startTime(), 0L) + ) + ); } } // Now add snapshots on disk that are not currently running @@ -230,20 +278,23 @@ private void buildResponse(SnapshotsInProgress snapshotsInProgress, } } - private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, - SnapshotsStatusRequest request, - List builder, - Set currentSnapshotNames, - String repositoryName, - CancellableTask task, - ActionListener listener) { + private void loadRepositoryData( + SnapshotsInProgress snapshotsInProgress, + SnapshotsStatusRequest request, + List builder, + Set currentSnapshotNames, + String repositoryName, + CancellableTask task, + ActionListener listener + ) { final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); final StepListener repositoryDataListener = new StepListener<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); final Collection snapshotIdsToLoad = new ArrayList<>(); repositoryDataListener.whenComplete(repositoryData -> { ensureNotCancelled(task); - final Map matchedSnapshotIds = repositoryData.getSnapshotIds().stream() + final Map matchedSnapshotIds = repositoryData.getSnapshotIds() + .stream() .filter(s -> requestedSnapshotNames.contains(s.getName())) .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); for (final String snapshotName : request.snapshots()) { @@ -256,8 +307,11 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, // neither in the current snapshot entries nor found in the repository if (request.ignoreUnavailable()) { // ignoring unavailable snapshots, so skip over - logger.debug("snapshot status request ignoring snapshot [{}], not found in repository [{}]", - snapshotName, repositoryName); + logger.debug( + "snapshot status request ignoring snapshot [{}], not found in repository [{}]", + snapshotName, + repositoryName + ); continue; } else { throw new SnapshotMissingException(repositoryName, snapshotName); @@ -273,47 +327,50 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, } else { final List threadSafeBuilder = Collections.synchronizedList(builder); repositoriesService.repository(repositoryName) - .getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, () -> false, - (context, snapshotInfo) -> { - List shardStatusBuilder = new ArrayList<>(); - final Map shardStatuses; - try { - shardStatuses = snapshotShards(repositoryName, repositoryData, task, snapshotInfo); - } catch (Exception e) { - // stops all further fetches of snapshotInfo since context is fail-fast - context.onFailure(e); - return; - } - for (Map.Entry shardStatus : shardStatuses.entrySet()) { - IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); - shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); - } - final SnapshotsInProgress.State state; - switch (snapshotInfo.state()) { - case FAILED: - state = SnapshotsInProgress.State.FAILED; - break; - case SUCCESS: - case PARTIAL: - // Translating both PARTIAL and SUCCESS to SUCCESS for now - // TODO: add the differentiation on the metadata level in the next major release - state = SnapshotsInProgress.State.SUCCESS; - break; - default: - throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); - } - final long startTime = snapshotInfo.startTime(); - final long endTime = snapshotInfo.endTime(); - assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) - : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; - threadSafeBuilder.add(new SnapshotStatus(new Snapshot(repositoryName, snapshotInfo.snapshotId()), state, - Collections.unmodifiableList(shardStatusBuilder), snapshotInfo.includeGlobalState(), - startTime, - // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 - (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime)); - }, - listener.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))) - )); + .getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, () -> false, (context, snapshotInfo) -> { + List shardStatusBuilder = new ArrayList<>(); + final Map shardStatuses; + try { + shardStatuses = snapshotShards(repositoryName, repositoryData, task, snapshotInfo); + } catch (Exception e) { + // stops all further fetches of snapshotInfo since context is fail-fast + context.onFailure(e); + return; + } + for (Map.Entry shardStatus : shardStatuses.entrySet()) { + IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); + shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); + } + final SnapshotsInProgress.State state; + switch (snapshotInfo.state()) { + case FAILED: + state = SnapshotsInProgress.State.FAILED; + break; + case SUCCESS: + case PARTIAL: + // Translating both PARTIAL and SUCCESS to SUCCESS for now + // TODO: add the differentiation on the metadata level in the next major release + state = SnapshotsInProgress.State.SUCCESS; + break; + default: + throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); + } + final long startTime = snapshotInfo.startTime(); + final long endTime = snapshotInfo.endTime(); + assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) + : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; + threadSafeBuilder.add( + new SnapshotStatus( + new Snapshot(repositoryName, snapshotInfo.snapshotId()), + state, + Collections.unmodifiableList(shardStatusBuilder), + snapshotInfo.includeGlobalState(), + startTime, + // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 + (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime + ) + ); + }, listener.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder))))); } }, listener::onFailure); } @@ -330,10 +387,12 @@ private void loadRepositoryData(SnapshotsInProgress snapshotsInProgress, * @param snapshotInfo snapshot info * @return map of shard id to snapshot status */ - private Map snapshotShards(final String repositoryName, - final RepositoryData repositoryData, - final CancellableTask task, - final SnapshotInfo snapshotInfo) throws IOException { + private Map snapshotShards( + final String repositoryName, + final RepositoryData repositoryData, + final CancellableTask task, + final SnapshotInfo snapshotInfo + ) throws IOException { final Repository repository = repositoriesService.repository(repositoryName); final Map shardStatus = new HashMap<>(); for (String index : snapshotInfo.indices()) { @@ -354,16 +413,13 @@ private Map snapshotShards(final String repos // not have an exception, it means that partial snapshots // were disabled and in this case, the shard snapshot will // *not* have any metadata, so attempting to read the shard - // snapshot status will throw an exception. Instead, we create + // snapshot status will throw an exception. Instead, we create // a status for the shard to indicate that the shard snapshot // could not be taken due to partial being set to false. shardSnapshotStatus = IndexShardSnapshotStatus.newFailed("skipped"); } else { ensureNotCancelled(task); - shardSnapshotStatus = repository.getShardSnapshotStatus( - snapshotInfo.snapshotId(), - indexId, - shardId); + shardSnapshotStatus = repository.getShardSnapshotStatus(snapshotInfo.snapshotId(), indexId, shardId); } shardStatus.put(shardId, shardSnapshotStatus); } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreException.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreException.java index 77f3df61c7acc..0d2369cb88554 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreException.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreException.java @@ -27,7 +27,7 @@ public IndexShardRestoreException(ShardId shardId, String msg, Throwable cause) setShard(shardId); } - public IndexShardRestoreException(StreamInput in) throws IOException{ + public IndexShardRestoreException(StreamInput in) throws IOException { super(in); } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreFailedException.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreFailedException.java index b08b0dbb7d87a..2f139aed9929b 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreFailedException.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardRestoreFailedException.java @@ -25,7 +25,7 @@ public IndexShardRestoreFailedException(ShardId shardId, String msg, Throwable c super(shardId, msg, cause); } - public IndexShardRestoreFailedException(StreamInput in) throws IOException{ + public IndexShardRestoreFailedException(StreamInput in) throws IOException { super(in); } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotException.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotException.java index 30b4e404cee23..30f1d8a826003 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotException.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotException.java @@ -27,7 +27,7 @@ public IndexShardSnapshotException(ShardId shardId, String msg, Throwable cause) setShard(shardId); } - public IndexShardSnapshotException(StreamInput in) throws IOException{ + public IndexShardSnapshotException(StreamInput in) throws IOException { super(in); } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotFailedException.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotFailedException.java index cfdb723e283fb..031657179e3db 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotFailedException.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotFailedException.java @@ -25,7 +25,7 @@ public IndexShardSnapshotFailedException(ShardId shardId, String msg, Throwable super(shardId, msg, cause); } - public IndexShardSnapshotFailedException(StreamInput in) throws IOException{ + public IndexShardSnapshotFailedException(StreamInput in) throws IOException { super(in); } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java index 0637aed3bb230..15519f0f5ad86 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java @@ -63,10 +63,19 @@ public enum Stage { private long indexVersion; private String failure; - private IndexShardSnapshotStatus(final Stage stage, final long startTime, final long totalTime, - final int incrementalFileCount, final int totalFileCount, final int processedFileCount, - final long incrementalSize, final long totalSize, final long processedSize, final String failure, - final String generation) { + private IndexShardSnapshotStatus( + final Stage stage, + final long startTime, + final long totalTime, + final int incrementalFileCount, + final int totalFileCount, + final int processedFileCount, + final long incrementalSize, + final long totalSize, + final long processedSize, + final String failure, + final String generation + ) { this.stage = new AtomicReference<>(Objects.requireNonNull(stage)); this.generation = new AtomicReference<>(generation); this.shardSnapshotResult = new AtomicReference<>(); @@ -81,8 +90,13 @@ private IndexShardSnapshotStatus(final Stage stage, final long startTime, final this.failure = failure; } - public synchronized Copy moveToStarted(final long startTime, final int incrementalFileCount, final int totalFileCount, - final long incrementalSize, final long totalSize) { + public synchronized Copy moveToStarted( + final long startTime, + final int incrementalFileCount, + final int totalFileCount, + final long incrementalSize, + final long totalSize + ) { if (stage.compareAndSet(Stage.INIT, Stage.STARTED)) { this.startTime = startTime; this.incrementalFileCount = incrementalFileCount; @@ -93,8 +107,9 @@ public synchronized Copy moveToStarted(final long startTime, final int increment throw new AbortedSnapshotException(); } else { assert false : "Should not try to move stage [" + stage.get() + "] to [STARTED]"; - throw new IllegalStateException("Unable to move the shard snapshot status to [STARTED]: " + - "expecting [INIT] but got [" + stage.get() + "]"); + throw new IllegalStateException( + "Unable to move the shard snapshot status to [STARTED]: " + "expecting [INIT] but got [" + stage.get() + "]" + ); } return asCopy(); } @@ -106,8 +121,9 @@ public synchronized Copy moveToFinalize(final long indexVersion) { throw new AbortedSnapshotException(); } else { assert false : "Should not try to move stage [" + stage.get() + "] to [FINALIZE]"; - throw new IllegalStateException("Unable to move the shard snapshot status to [FINALIZE]: " + - "expecting [STARTED] but got [" + stage.get() + "]"); + throw new IllegalStateException( + "Unable to move the shard snapshot status to [FINALIZE]: " + "expecting [STARTED] but got [" + stage.get() + "]" + ); } return asCopy(); } @@ -121,8 +137,9 @@ public synchronized void moveToDone(final long endTime, final ShardSnapshotResul this.generation.set(shardSnapshotResult.getGeneration()); } else { assert false : "Should not try to move stage [" + stage.get() + "] to [DONE]"; - throw new IllegalStateException("Unable to move the shard snapshot status to [DONE]: " + - "expecting [FINALIZE] but got [" + stage.get() + "]"); + throw new IllegalStateException( + "Unable to move the shard snapshot status to [DONE]: " + "expecting [FINALIZE] but got [" + stage.get() + "]" + ); } } @@ -167,10 +184,19 @@ public synchronized void addProcessedFile(long size) { * @return a {@link IndexShardSnapshotStatus.Copy} */ public synchronized IndexShardSnapshotStatus.Copy asCopy() { - return new IndexShardSnapshotStatus.Copy(stage.get(), startTime, totalTime, - incrementalFileCount, totalFileCount, processedFileCount, - incrementalSize, totalSize, processedSize, - indexVersion, failure); + return new IndexShardSnapshotStatus.Copy( + stage.get(), + startTime, + totalTime, + incrementalFileCount, + totalFileCount, + processedFileCount, + incrementalSize, + totalSize, + processedSize, + indexVersion, + failure + ); } public static IndexShardSnapshotStatus newInitializing(String generation) { @@ -185,12 +211,29 @@ public static IndexShardSnapshotStatus newFailed(final String failure) { return new IndexShardSnapshotStatus(Stage.FAILURE, 0L, 0L, 0, 0, 0, 0, 0, 0, failure, null); } - public static IndexShardSnapshotStatus newDone(final long startTime, final long totalTime, - final int incrementalFileCount, final int fileCount, - final long incrementalSize, final long size, String generation) { + public static IndexShardSnapshotStatus newDone( + final long startTime, + final long totalTime, + final int incrementalFileCount, + final int fileCount, + final long incrementalSize, + final long size, + String generation + ) { // The snapshot is done which means the number of processed files is the same as total - return new IndexShardSnapshotStatus(Stage.DONE, startTime, totalTime, incrementalFileCount, fileCount, incrementalFileCount, - incrementalSize, size, incrementalSize, null, generation); + return new IndexShardSnapshotStatus( + Stage.DONE, + startTime, + totalTime, + incrementalFileCount, + fileCount, + incrementalFileCount, + incrementalSize, + size, + incrementalSize, + null, + generation + ); } /** @@ -210,10 +253,19 @@ public static class Copy { private final long indexVersion; private final String failure; - public Copy(final Stage stage, final long startTime, final long totalTime, - final int incrementalFileCount, final int totalFileCount, final int processedFileCount, - final long incrementalSize, final long totalSize, final long processedSize, - final long indexVersion, final String failure) { + public Copy( + final Stage stage, + final long startTime, + final long totalTime, + final int incrementalFileCount, + final int totalFileCount, + final int processedFileCount, + final long incrementalSize, + final long totalSize, + final long processedSize, + final long indexVersion, + final String failure + ) { this.stage = stage; this.startTime = startTime; this.totalTime = totalTime; @@ -273,19 +325,31 @@ public String getFailure() { @Override public String toString() { - return "index shard snapshot status (" + - "stage=" + stage + - ", startTime=" + startTime + - ", totalTime=" + totalTime + - ", incrementalFileCount=" + incrementalFileCount + - ", totalFileCount=" + totalFileCount + - ", processedFileCount=" + processedFileCount + - ", incrementalSize=" + incrementalSize + - ", totalSize=" + totalSize + - ", processedSize=" + processedSize + - ", indexVersion=" + indexVersion + - ", failure='" + failure + '\'' + - ')'; + return "index shard snapshot status (" + + "stage=" + + stage + + ", startTime=" + + startTime + + ", totalTime=" + + totalTime + + ", incrementalFileCount=" + + incrementalFileCount + + ", totalFileCount=" + + totalFileCount + + ", processedFileCount=" + + processedFileCount + + ", incrementalSize=" + + incrementalSize + + ", totalSize=" + + totalSize + + ", processedSize=" + + processedSize + + ", indexVersion=" + + indexVersion + + ", failure='" + + failure + + '\'' + + ')'; } } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index be5ef119517b5..1b6fe2a1ebded 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -63,7 +63,7 @@ public FileInfo(String name, StoreFileMetadata metadata, ByteSizeValue partSize) numberOfParts = 1; } else { long longNumberOfParts = 1L + (metadata.length() - 1L) / partBytes; // ceil(len/partBytes), but beware of long overflow - numberOfParts = (int)longNumberOfParts; + numberOfParts = (int) longNumberOfParts; if (numberOfParts != longNumberOfParts) { // also beware of int overflow, although 2^32 parts is already ludicrous throw new IllegalArgumentException("part size [" + partSize + "] too small for file [" + metadata + "]"); } @@ -322,11 +322,17 @@ public static FileInfo fromXContent(XContentParser parser) throws IOException { @Override public String toString() { - return "[name: " + name + - ", numberOfParts: " + numberOfParts + - ", partSize: " + partSize + - ", partBytes: " + partBytes + - ", metadata: " + metadata + "]"; + return "[name: " + + name + + ", numberOfParts: " + + numberOfParts + + ", partSize: " + + partSize + + ", partBytes: " + + partBytes + + ", metadata: " + + metadata + + "]"; } } @@ -358,10 +364,14 @@ public String toString() { * @param incrementalFileCount incremental of files that were snapshotted * @param incrementalSize incremental size of snapshot */ - public BlobStoreIndexShardSnapshot(String snapshot, long indexVersion, List indexFiles, - long startTime, long time, - int incrementalFileCount, - long incrementalSize + public BlobStoreIndexShardSnapshot( + String snapshot, + long indexVersion, + List indexFiles, + long startTime, + long time, + int incrementalFileCount, + long incrementalSize ) { assert snapshot != null; assert indexVersion >= 0; @@ -460,7 +470,6 @@ public static long totalSize(List indexFiles) { private static final String INCREMENTAL_FILE_COUNT = "number_of_files"; private static final String INCREMENTAL_SIZE = "total_size"; - private static final ParseField PARSE_NAME = new ParseField(NAME); private static final ParseField PARSE_INDEX_VERSION = new ParseField(INDEX_VERSION, "index-version"); private static final ParseField PARSE_START_TIME = new ParseField(START_TIME); @@ -545,7 +554,14 @@ public static BlobStoreIndexShardSnapshot fromXContent(XContentParser parser) th } } - return new BlobStoreIndexShardSnapshot(snapshot, indexVersion, Collections.unmodifiableList(indexFiles), - startTime, time, incrementalFileCount, incrementalSize); + return new BlobStoreIndexShardSnapshot( + snapshot, + indexVersion, + Collections.unmodifiableList(indexFiles), + startTime, + time, + incrementalFileCount, + incrementalSize + ); } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index 76ad67ea3248b..7658c4aef4e9b 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -261,8 +261,8 @@ public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) t while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); - if (ParseFields.FILES.match(currentFieldName, parser.getDeprecationHandler()) && - parser.nextToken() == XContentParser.Token.START_ARRAY) { + if (ParseFields.FILES.match(currentFieldName, parser.getDeprecationHandler()) + && parser.nextToken() == XContentParser.Token.START_ARRAY) { List fileNames = new ArrayList<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { fileNames.add(parser.text()); @@ -288,8 +288,9 @@ public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) t assert fileInfo != null; fileInfosBuilder.add(fileInfo); } - snapshots.add(new SnapshotFiles(entry.getKey(), Collections.unmodifiableList(fileInfosBuilder), - historyUUIDs.get(entry.getKey()))); + snapshots.add( + new SnapshotFiles(entry.getKey(), Collections.unmodifiableList(fileInfosBuilder), historyUUIDs.get(entry.getKey())) + ); } return new BlobStoreIndexShardSnapshots(files, Collections.unmodifiableList(snapshots)); } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java index 09cb966ec8718..f25eba639b39b 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java @@ -109,7 +109,7 @@ public boolean containPhysicalIndexFile(String physicalName) { private FileInfo findPhysicalIndexFile(String physicalName) { if (physicalFiles == null) { Map files = new HashMap<>(); - for(FileInfo fileInfo : indexFiles) { + for (FileInfo fileInfo : indexFiles) { files.put(fileInfo.physicalName(), fileInfo); } this.physicalFiles = files; @@ -123,7 +123,12 @@ public long totalSize() { @Override public String toString() { - return "SnapshotFiles{snapshot=[" + snapshot + "], shardStateIdentifier=[" + shardStateIdentifier + "], indexFiles=" + indexFiles - + "}"; + return "SnapshotFiles{snapshot=[" + + snapshot + + "], shardStateIdentifier=[" + + shardStateIdentifier + + "], indexFiles=" + + indexFiles + + "}"; } } diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 12351f2cb0aed..3a87611e27dfc 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -69,16 +69,33 @@ public void getRepositoryData(ActionListener listener) { } @Override - public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryStateId, Metadata clusterMetadata, - SnapshotInfo snapshotInfo, Version repositoryMetaVersion, - Function stateTransformer, ActionListener listener) { - in.finalizeSnapshot(shardGenerations, repositoryStateId, clusterMetadata, snapshotInfo, repositoryMetaVersion, stateTransformer, - listener); - } - - @Override - public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + public void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ) { + in.finalizeSnapshot( + shardGenerations, + repositoryStateId, + clusterMetadata, + snapshotInfo, + repositoryMetaVersion, + stateTransformer, + listener + ); + } + + @Override + public void deleteSnapshots( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + ActionListener listener + ) { in.deleteSnapshots(snapshotIds, repositoryStateId, repositoryMetaVersion, listener); } @@ -116,9 +133,16 @@ public boolean isReadOnly() { public void snapshotShard(SnapshotShardContext context) { in.snapshotShard(context); } + @Override - public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState, - ActionListener listener) { + public void restoreShard( + Store store, + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId, + RecoveryState recoveryState, + ActionListener listener + ) { in.restoreShard(store, snapshotId, indexId, snapshotShardId, recoveryState, listener); } @@ -138,14 +162,22 @@ public void updateState(ClusterState state) { } @Override - public void executeConsistentStateUpdate(Function createUpdateTask, String source, - Consumer onFailure) { + public void executeConsistentStateUpdate( + Function createUpdateTask, + String source, + Consumer onFailure + ) { in.executeConsistentStateUpdate(createUpdateTask, source, onFailure); } @Override - public void cloneShardSnapshot(SnapshotId source, SnapshotId target, RepositoryShardId shardId, String shardGeneration, - ActionListener listener) { + public void cloneShardSnapshot( + SnapshotId source, + SnapshotId target, + RepositoryShardId shardId, + String shardGeneration, + ActionListener listener + ) { in.cloneShardSnapshot(source, target, shardId, shardGeneration, listener); } diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java index 750f32851fa34..1d746ae82a25c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -57,11 +57,13 @@ public final class GetSnapshotInfoContext implements ActionListener snapshotIds, - boolean abortOnFailure, - BooleanSupplier isCancelled, - BiConsumer onSnapshotInfo, - ActionListener doneListener) { + public GetSnapshotInfoContext( + Collection snapshotIds, + boolean abortOnFailure, + BooleanSupplier isCancelled, + BiConsumer onSnapshotInfo, + ActionListener doneListener + ) { assert snapshotIds.isEmpty() == false : "no snapshot ids to fetch given"; this.snapshotIds = List.copyOf(snapshotIds); this.counter = new CountDown(snapshotIds.size()); diff --git a/server/src/main/java/org/elasticsearch/repositories/IndexMetaDataGenerations.java b/server/src/main/java/org/elasticsearch/repositories/IndexMetaDataGenerations.java index b282a4129b532..87b102acc3cea 100644 --- a/server/src/main/java/org/elasticsearch/repositories/IndexMetaDataGenerations.java +++ b/server/src/main/java/org/elasticsearch/repositories/IndexMetaDataGenerations.java @@ -42,8 +42,8 @@ public final class IndexMetaDataGenerations { final Map identifiers; IndexMetaDataGenerations(Map> lookup, Map identifiers) { - assert identifiers.keySet().equals(lookup.values().stream().flatMap(m -> m.values().stream()).collect(Collectors.toSet())) : - "identifier mappings " + identifiers + " don't track the same blob ids as the lookup map " + lookup; + assert identifiers.keySet().equals(lookup.values().stream().flatMap(m -> m.values().stream()).collect(Collectors.toSet())) + : "identifier mappings " + identifiers + " don't track the same blob ids as the lookup map " + lookup; assert lookup.values().stream().noneMatch(Map::isEmpty) : "Lookup contained empty map [" + lookup + "]"; this.lookup = Map.copyOf(lookup); this.identifiers = Map.copyOf(identifiers); @@ -92,8 +92,11 @@ public String indexMetaBlobId(SnapshotId snapshotId, IndexId indexId) { * @param newIdentifiers new mappings of index metadata identifier to blob id * @return instance with added snapshot */ - public IndexMetaDataGenerations withAddedSnapshot(SnapshotId snapshotId, Map newLookup, - Map newIdentifiers) { + public IndexMetaDataGenerations withAddedSnapshot( + SnapshotId snapshotId, + Map newLookup, + Map newIdentifiers + ) { final Map> updatedIndexMetaLookup = new HashMap<>(this.lookup); final Map updatedIndexMetaIdentifiers = new HashMap<>(identifiers); updatedIndexMetaIdentifiers.putAll(newIdentifiers); @@ -123,8 +126,8 @@ public IndexMetaDataGenerations withRemovedSnapshots(Collection snap final Map> updatedIndexMetaLookup = new HashMap<>(lookup); updatedIndexMetaLookup.keySet().removeAll(snapshotIds); final Map updatedIndexMetaIdentifiers = new HashMap<>(identifiers); - updatedIndexMetaIdentifiers.keySet().removeIf( - k -> updatedIndexMetaLookup.values().stream().noneMatch(identifiers -> identifiers.containsValue(k))); + updatedIndexMetaIdentifiers.keySet() + .removeIf(k -> updatedIndexMetaLookup.values().stream().noneMatch(identifiers -> identifiers.containsValue(k))); return new IndexMetaDataGenerations(updatedIndexMetaLookup, updatedIndexMetaIdentifiers); } @@ -159,9 +162,14 @@ public String toString() { * @return identifier string */ public static String buildUniqueIdentifier(IndexMetadata indexMetaData) { - return indexMetaData.getIndexUUID() + - "-" + indexMetaData.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID, IndexMetadata.INDEX_UUID_NA_VALUE) + - "-" + indexMetaData.getSettingsVersion() + "-" + indexMetaData.getMappingVersion() + - "-" + indexMetaData.getAliasesVersion(); + return indexMetaData.getIndexUUID() + + "-" + + indexMetaData.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID, IndexMetadata.INDEX_UUID_NA_VALUE) + + "-" + + indexMetaData.getSettingsVersion() + + "-" + + indexMetaData.getMappingVersion() + + "-" + + indexMetaData.getAliasesVersion(); } } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java index 921fddc731627..f7931bdb54a5d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java @@ -30,16 +30,29 @@ public final class RepositoriesModule { private final RepositoriesService repositoriesService; - public RepositoriesModule(Environment env, List repoPlugins, TransportService transportService, - ClusterService clusterService, BigArrays bigArrays, NamedXContentRegistry namedXContentRegistry, - RecoverySettings recoverySettings) { + public RepositoriesModule( + Environment env, + List repoPlugins, + TransportService transportService, + ClusterService clusterService, + BigArrays bigArrays, + NamedXContentRegistry namedXContentRegistry, + RecoverySettings recoverySettings + ) { Map factories = new HashMap<>(); - factories.put(FsRepository.TYPE, metadata -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, - recoverySettings)); + factories.put( + FsRepository.TYPE, + metadata -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) + ); for (RepositoryPlugin repoPlugin : repoPlugins) { - Map newRepoTypes = repoPlugin.getRepositories(env, namedXContentRegistry, clusterService, - bigArrays, recoverySettings); + Map newRepoTypes = repoPlugin.getRepositories( + env, + namedXContentRegistry, + clusterService, + bigArrays, + recoverySettings + ); for (Map.Entry entry : newRepoTypes.entrySet()) { if (factories.put(entry.getKey(), entry.getValue()) != null) { throw new IllegalArgumentException("Repository type [" + entry.getKey() + "] is already registered"); @@ -49,15 +62,20 @@ public RepositoriesModule(Environment env, List repoPlugins, T Map internalFactories = new HashMap<>(); for (RepositoryPlugin repoPlugin : repoPlugins) { - Map newRepoTypes = repoPlugin.getInternalRepositories(env, namedXContentRegistry, clusterService, - recoverySettings); + Map newRepoTypes = repoPlugin.getInternalRepositories( + env, + namedXContentRegistry, + clusterService, + recoverySettings + ); for (Map.Entry entry : newRepoTypes.entrySet()) { if (internalFactories.put(entry.getKey(), entry.getValue()) != null) { throw new IllegalArgumentException("Internal repository type [" + entry.getKey() + "] is already registered"); } if (factories.put(entry.getKey(), entry.getValue()) != null) { - throw new IllegalArgumentException("Internal repository type [" + entry.getKey() + "] is already registered as a " + - "non-internal repository"); + throw new IllegalArgumentException( + "Internal repository type [" + entry.getKey() + "] is already registered as a " + "non-internal repository" + ); } } } @@ -65,8 +83,14 @@ public RepositoriesModule(Environment env, List repoPlugins, T Settings settings = env.settings(); Map repositoryTypes = Collections.unmodifiableMap(factories); Map internalRepositoryTypes = Collections.unmodifiableMap(internalFactories); - repositoriesService = new RepositoriesService(settings, clusterService, transportService, repositoryTypes, - internalRepositoryTypes, transportService.getThreadPool()); + repositoriesService = new RepositoriesService( + settings, + clusterService, + transportService, + repositoryTypes, + internalRepositoryTypes, + transportService.getThreadPool() + ); } public RepositoriesService getRepositoryService() { diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index 1255b05c6cad3..610b54f64758b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -68,11 +68,18 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C private static final Logger logger = LogManager.getLogger(RepositoriesService.class); - public static final Setting REPOSITORIES_STATS_ARCHIVE_RETENTION_PERIOD = - Setting.positiveTimeSetting("repositories.stats.archive.retention_period", TimeValue.timeValueHours(2), Setting.Property.NodeScope); - - public static final Setting REPOSITORIES_STATS_ARCHIVE_MAX_ARCHIVED_STATS = - Setting.intSetting("repositories.stats.archive.max_archived_stats", 100, 0, Setting.Property.NodeScope); + public static final Setting REPOSITORIES_STATS_ARCHIVE_RETENTION_PERIOD = Setting.positiveTimeSetting( + "repositories.stats.archive.retention_period", + TimeValue.timeValueHours(2), + Setting.Property.NodeScope + ); + + public static final Setting REPOSITORIES_STATS_ARCHIVE_MAX_ARCHIVED_STATS = Setting.intSetting( + "repositories.stats.archive.max_archived_stats", + 100, + 0, + Setting.Property.NodeScope + ); public static final String SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY = "index.store.snapshot.repository_name"; public static final String SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY = "index.store.snapshot.repository_uuid"; @@ -90,10 +97,14 @@ public class RepositoriesService extends AbstractLifecycleComponent implements C private volatile Map repositories = Collections.emptyMap(); private final RepositoriesStatsArchive repositoriesStatsArchive; - - public RepositoriesService(Settings settings, ClusterService clusterService, TransportService transportService, - Map typesRegistry, Map internalTypesRegistry, - ThreadPool threadPool) { + public RepositoriesService( + Settings settings, + ClusterService clusterService, + TransportService transportService, + Map typesRegistry, + Map internalTypesRegistry, + ThreadPool threadPool + ) { this.typesRegistry = typesRegistry; this.internalTypesRegistry = internalTypesRegistry; this.clusterService = clusterService; @@ -106,9 +117,11 @@ public RepositoriesService(Settings settings, ClusterService clusterService, Tra } } this.verifyAction = new VerifyNodeRepositoryAction(transportService, clusterService, this); - this.repositoriesStatsArchive = new RepositoriesStatsArchive(REPOSITORIES_STATS_ARCHIVE_RETENTION_PERIOD.get(settings), + this.repositoriesStatsArchive = new RepositoriesStatsArchive( + REPOSITORIES_STATS_ARCHIVE_RETENTION_PERIOD.get(settings), REPOSITORIES_STATS_ARCHIVE_MAX_ARCHIVED_STATS.get(settings), - threadPool::relativeTimeInMillis); + threadPool::relativeTimeInMillis + ); } /** @@ -154,25 +167,28 @@ public void registerRepository(final PutRepositoryRequest request, final ActionL // When verification has completed, get the repository data for the first time final StepListener getRepositoryDataStep = new StepListener<>(); - verifyStep.whenComplete(ignored -> threadPool.generic().execute( - ActionRunnable.wrap(getRepositoryDataStep, l -> repository(request.name()).getRepositoryData(l))), listener::onFailure); + verifyStep.whenComplete( + ignored -> threadPool.generic() + .execute(ActionRunnable.wrap(getRepositoryDataStep, l -> repository(request.name()).getRepositoryData(l))), + listener::onFailure + ); // When the repository metadata is ready, update the repository UUID stored in the cluster state, if available final StepListener updateRepoUuidStep = new StepListener<>(); getRepositoryDataStep.whenComplete( repositoryData -> updateRepositoryUuidInMetadata(clusterService, request.name(), repositoryData, updateRepoUuidStep), - listener::onFailure); + listener::onFailure + ); // Finally respond to the outer listener with the response from the original cluster state update - updateRepoUuidStep.whenComplete( - ignored -> acknowledgementStep.addListener(listener), - listener::onFailure); + updateRepoUuidStep.whenComplete(ignored -> acknowledgementStep.addListener(listener), listener::onFailure); } else { acknowledgementStep.addListener(listener); } - clusterService.submitStateUpdateTask("put_repository [" + request.name() + "]", + clusterService.submitStateUpdateTask( + "put_repository [" + request.name() + "]", new AckedClusterStateUpdateTask(request, acknowledgementStep) { private boolean found = false; @@ -243,7 +259,8 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS } publicationStep.onResponse(null); } - }); + } + ); } /** @@ -257,7 +274,8 @@ public static void updateRepositoryUuidInMetadata( ClusterService clusterService, final String repositoryName, RepositoryData repositoryData, - ActionListener listener) { + ActionListener listener + ) { final String repositoryUuid = repositoryData.getUuid(); if (repositoryUuid.equals(RepositoryData.MISSING_UUID)) { @@ -265,28 +283,30 @@ public static void updateRepositoryUuidInMetadata( return; } - final RepositoriesMetadata currentReposMetadata - = clusterService.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final RepositoriesMetadata currentReposMetadata = clusterService.state() + .metadata() + .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); final RepositoryMetadata repositoryMetadata = currentReposMetadata.repository(repositoryName); if (repositoryMetadata == null || repositoryMetadata.uuid().equals(repositoryUuid)) { listener.onResponse(null); return; } - clusterService.submitStateUpdateTask("update repository UUID [" + repositoryName + "] to [" + repositoryUuid + "]", + clusterService.submitStateUpdateTask( + "update repository UUID [" + repositoryName + "] to [" + repositoryUuid + "]", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - final RepositoriesMetadata currentReposMetadata - = currentState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final RepositoriesMetadata currentReposMetadata = currentState.metadata() + .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); final RepositoryMetadata repositoryMetadata = currentReposMetadata.repository(repositoryName); if (repositoryMetadata == null || repositoryMetadata.uuid().equals(repositoryUuid)) { return currentState; } else { final RepositoriesMetadata newReposMetadata = currentReposMetadata.withUuid(repositoryName, repositoryUuid); - final Metadata.Builder metadata - = Metadata.builder(currentState.metadata()).putCustom(RepositoriesMetadata.TYPE, newReposMetadata); + final Metadata.Builder metadata = Metadata.builder(currentState.metadata()) + .putCustom(RepositoriesMetadata.TYPE, newReposMetadata); return ClusterState.builder(currentState).metadata(metadata).build(); } } @@ -300,7 +320,8 @@ public void onFailure(String source, Exception e) { public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { listener.onResponse(null); } - }); + } + ); } /** @@ -312,7 +333,8 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * @param listener unregister repository listener */ public void unregisterRepository(final DeleteRepositoryRequest request, final ActionListener listener) { - clusterService.submitStateUpdateTask("delete_repository [" + request.name() + "]", + clusterService.submitStateUpdateTask( + "delete_repository [" + request.name() + "]", new AckedClusterStateUpdateTask(request, listener) { private final List deletedRepositories = new ArrayList<>(); @@ -359,7 +381,8 @@ public boolean mustAck(DiscoveryNode discoveryNode) { // repository was created on both master and data nodes return discoveryNode.isMasterNode() || discoveryNode.canContainData(); } - }); + } + ); } public void verifyRepository(final String repositoryName, final ActionListener> listener) { @@ -370,26 +393,35 @@ protected void doRun() { final String verificationToken = repository.startVerification(); if (verificationToken != null) { try { - verifyAction.verify(repositoryName, verificationToken, listener.delegateFailure( - (delegatedListener, verifyResponse) -> threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { - try { - repository.endVerification(verificationToken); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage( - "[{}] failed to finish repository verification", repositoryName), e); - delegatedListener.onFailure(e); - return; - } - delegatedListener.onResponse(verifyResponse); - }))); + verifyAction.verify( + repositoryName, + verificationToken, + listener.delegateFailure( + (delegatedListener, verifyResponse) -> threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { + try { + repository.endVerification(verificationToken); + } catch (Exception e) { + logger.warn( + () -> new ParameterizedMessage("[{}] failed to finish repository verification", repositoryName), + e + ); + delegatedListener.onFailure(e); + return; + } + delegatedListener.onResponse(verifyResponse); + }) + ) + ); } catch (Exception e) { threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { try { repository.endVerification(verificationToken); } catch (Exception inner) { inner.addSuppressed(e); - logger.warn(() -> new ParameterizedMessage( - "[{}] failed to finish repository verification", repositoryName), inner); + logger.warn( + () -> new ParameterizedMessage("[{}] failed to finish repository verification", repositoryName), + inner + ); } listener.onFailure(e); }); @@ -417,8 +449,9 @@ public static boolean isDedicatedVotingOnlyNode(Set roles) { public void applyClusterState(ClusterChangedEvent event) { try { final ClusterState state = event.state(); - final RepositoriesMetadata oldMetadata = - event.previousState().getMetadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final RepositoriesMetadata oldMetadata = event.previousState() + .getMetadata() + .custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); final RepositoriesMetadata newMetadata = state.getMetadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); // Check if repositories got changed @@ -462,8 +495,7 @@ public void applyClusterState(ClusterChangedEvent event) { } catch (RepositoryException ex) { // TODO: this catch is bogus, it means the old repo is already closed, // but we have nothing to replace it - logger.warn(() -> new ParameterizedMessage("failed to change repository [{}]", - repositoryMetadata.name()), ex); + logger.warn(() -> new ParameterizedMessage("failed to change repository [{}]", repositoryMetadata.name()), ex); } } } else { @@ -564,11 +596,26 @@ public void registerInternalRepository(String name, String type) { return createRepository(metadata, internalTypesRegistry); }); if (type.equals(repository.getMetadata().type()) == false) { - logger.warn(new ParameterizedMessage("internal repository [{}][{}] already registered. this prevented the registration of " + - "internal repository [{}][{}].", name, repository.getMetadata().type(), name, type)); + logger.warn( + new ParameterizedMessage( + "internal repository [{}][{}] already registered. this prevented the registration of " + + "internal repository [{}][{}].", + name, + repository.getMetadata().type(), + name, + type + ) + ); } else if (repositories.containsKey(name)) { - logger.warn(new ParameterizedMessage("non-internal repository [{}] already registered. this repository will block the " + - "usage of internal repository [{}][{}].", name, metadata.type(), name)); + logger.warn( + new ParameterizedMessage( + "non-internal repository [{}] already registered. this repository will block the " + + "usage of internal repository [{}][{}].", + name, + metadata.type(), + name + ) + ); } } @@ -605,8 +652,7 @@ private Repository createRepository(RepositoryMetadata repositoryMetadata, Map indices.size() ? ",..." : "") + throw newRepositoryInUseException( + repositoryMetadata.name(), + "found " + + count + + " searchable snapshots indices that use the repository: " + + Strings.collectionToCommaDelimitedString(indices) + + (count > indices.size() ? ",..." : "") ); } } @@ -696,11 +750,8 @@ private static boolean indexSettingsMatchRepositoryMetadata(Settings indexSettin } private static IllegalStateException newRepositoryInUseException(String repository, String reason) { - return new IllegalStateException("trying to modify or unregister repository [" - + repository - + "] that is currently used (" - + reason - + ')' + return new IllegalStateException( + "trying to modify or unregister repository [" + repository + "] that is currently used (" + reason + ')' ); } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesStatsArchive.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesStatsArchive.java index 00281026103c6..05b5550e0f59c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesStatsArchive.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesStatsArchive.java @@ -28,9 +28,7 @@ public final class RepositoriesStatsArchive { private final LongSupplier relativeTimeSupplier; private final Deque archive = new ArrayDeque<>(); - public RepositoriesStatsArchive(TimeValue retentionPeriod, - int maxCapacity, - LongSupplier relativeTimeSupplier) { + public RepositoriesStatsArchive(TimeValue retentionPeriod, int maxCapacity, LongSupplier relativeTimeSupplier) { this.retentionPeriod = retentionPeriod; this.maxCapacity = maxCapacity; this.relativeTimeSupplier = relativeTimeSupplier; @@ -90,8 +88,11 @@ private void evict() { private boolean containsRepositoryStats(RepositoryStatsSnapshot repositoryStats) { return archive.stream() - .anyMatch(entry -> - entry.repositoryStatsSnapshot.getRepositoryInfo().ephemeralId.equals(repositoryStats.getRepositoryInfo().ephemeralId)); + .anyMatch( + entry -> entry.repositoryStatsSnapshot.getRepositoryInfo().ephemeralId.equals( + repositoryStats.getRepositoryInfo().ephemeralId + ) + ); } private static class ArchiveEntry { diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 20eb22231f33d..afdbd50c0e011 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -85,23 +85,23 @@ default Repository create(RepositoryMetadata metadata, Function listener) { getSnapshotInfo( - new GetSnapshotInfoContext( - List.of(snapshotId), - true, - () -> false, - (context, snapshotInfo) -> listener.onResponse(snapshotInfo), - new ActionListener<>() { - @Override - public void onResponse(Void o) { - // ignored - } + new GetSnapshotInfoContext( + List.of(snapshotId), + true, + () -> false, + (context, snapshotInfo) -> listener.onResponse(snapshotInfo), + new ActionListener<>() { + @Override + public void onResponse(Void o) { + // ignored + } - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - } - ) + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + ) ); } @@ -146,9 +146,15 @@ public void onFailure(Exception e) { * is used to remove any state tracked for the in-progress snapshot from the cluster state * @param listener listener to be invoked with the new {@link RepositoryData} after completing the snapshot */ - void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryStateId, Metadata clusterMetadata, - SnapshotInfo snapshotInfo, Version repositoryMetaVersion, Function stateTransformer, - ActionListener listener); + void finalizeSnapshot( + ShardGenerations shardGenerations, + long repositoryStateId, + Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + ActionListener listener + ); /** * Deletes snapshots @@ -158,8 +164,13 @@ void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryStateId, * @param repositoryMetaVersion version of the updated repository metadata to write * @param listener completion listener */ - void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener); + void deleteSnapshots( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + ActionListener listener + ); + /** * Returns snapshot throttle time in nanoseconds */ @@ -232,8 +243,15 @@ default RepositoryStats stats() { * @param recoveryState recovery state * @param listener listener to invoke once done */ - void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState, - ActionListener listener); + void restoreShard( + Store store, + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId, + RecoveryState recoveryState, + ActionListener listener + ); + /** * Retrieve shard snapshot status for the stored snapshot * @@ -276,8 +294,11 @@ default boolean canUpdateInPlace(Settings updatedSettings, Set ignoredSe * @param source the source of the cluster state update task * @param onFailure error handler invoked on failure to get a consistent view of the current {@link RepositoryData} */ - void executeConsistentStateUpdate(Function createUpdateTask, String source, - Consumer onFailure); + void executeConsistentStateUpdate( + Function createUpdateTask, + String source, + Consumer onFailure + ); /** * Clones a shard snapshot. @@ -289,11 +310,12 @@ void executeConsistentStateUpdate(Function listener); + SnapshotId source, + SnapshotId target, + RepositoryShardId shardId, + @Nullable String shardGeneration, + ActionListener listener + ); /** * Hook that allows a repository to filter the user supplied snapshot metadata in {@link SnapshotsInProgress.Entry#userMetadata()} diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryCleanupResult.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryCleanupResult.java index 09bb2337f7c63..4bda8ba72ae70 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryCleanupResult.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryCleanupResult.java @@ -21,8 +21,11 @@ public final class RepositoryCleanupResult implements Writeable, ToXContentObject { - public static final ObjectParser PARSER = - new ObjectParser<>(RepositoryCleanupResult.class.getName(), true, RepositoryCleanupResult::new); + public static final ObjectParser PARSER = new ObjectParser<>( + RepositoryCleanupResult.class.getName(), + true, + RepositoryCleanupResult::new + ); private static final String DELETED_BLOBS = "deleted_blobs"; diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java index ecc221606d38a..5977d0077772b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java @@ -68,15 +68,16 @@ public final class RepositoryData { * An instance initialized for an empty repository. */ public static final RepositoryData EMPTY = new RepositoryData( - MISSING_UUID, - EMPTY_REPO_GEN, - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), - ShardGenerations.EMPTY, - IndexMetaDataGenerations.EMPTY, - MISSING_UUID); + MISSING_UUID, + EMPTY_REPO_GEN, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + ShardGenerations.EMPTY, + IndexMetaDataGenerations.EMPTY, + MISSING_UUID + ); /** * A UUID that identifies this repository. @@ -124,36 +125,39 @@ public final class RepositoryData { private final ShardGenerations shardGenerations; public RepositoryData( - String uuid, - long genId, - Map snapshotIds, - Map snapshotsDetails, - Map> indexSnapshots, - ShardGenerations shardGenerations, - IndexMetaDataGenerations indexMetaDataGenerations, - String clusterUUID) { + String uuid, + long genId, + Map snapshotIds, + Map snapshotsDetails, + Map> indexSnapshots, + ShardGenerations shardGenerations, + IndexMetaDataGenerations indexMetaDataGenerations, + String clusterUUID + ) { this( - uuid, - genId, - Collections.unmodifiableMap(snapshotIds), - Collections.unmodifiableMap(snapshotsDetails), - indexSnapshots.keySet().stream().collect(Collectors.toUnmodifiableMap(IndexId::getName, Function.identity())), - Collections.unmodifiableMap(indexSnapshots), - shardGenerations, - indexMetaDataGenerations, - clusterUUID); + uuid, + genId, + Collections.unmodifiableMap(snapshotIds), + Collections.unmodifiableMap(snapshotsDetails), + indexSnapshots.keySet().stream().collect(Collectors.toUnmodifiableMap(IndexId::getName, Function.identity())), + Collections.unmodifiableMap(indexSnapshots), + shardGenerations, + indexMetaDataGenerations, + clusterUUID + ); } private RepositoryData( - String uuid, - long genId, - Map snapshotIds, - Map snapshotsDetails, - Map indices, - Map> indexSnapshots, - ShardGenerations shardGenerations, - IndexMetaDataGenerations indexMetaDataGenerations, - String clusterUUID) { + String uuid, + long genId, + Map snapshotIds, + Map snapshotsDetails, + Map indices, + Map> indexSnapshots, + ShardGenerations shardGenerations, + IndexMetaDataGenerations indexMetaDataGenerations, + String clusterUUID + ) { this.uuid = Objects.requireNonNull(uuid); this.genId = genId; this.snapshotIds = snapshotIds; @@ -163,25 +167,34 @@ private RepositoryData( this.shardGenerations = shardGenerations; this.indexMetaDataGenerations = indexMetaDataGenerations; this.clusterUUID = Objects.requireNonNull(clusterUUID); - assert uuid.equals(MISSING_UUID) == clusterUUID.equals(MISSING_UUID) : "Either repository- and cluster UUID must both be missing" + - " or neither of them must be missing but saw [" + uuid + "][" + clusterUUID + "]"; - assert indices.values().containsAll(shardGenerations.indices()) : "ShardGenerations contained indices " - + shardGenerations.indices() + " but snapshots only reference indices " + indices.values(); - assert indexSnapshots.values().stream().noneMatch(snapshotIdList -> Set.copyOf(snapshotIdList).size() != snapshotIdList.size()) : - "Found duplicate snapshot ids per index in [" + indexSnapshots + "]"; + assert uuid.equals(MISSING_UUID) == clusterUUID.equals(MISSING_UUID) + : "Either repository- and cluster UUID must both be missing" + + " or neither of them must be missing but saw [" + + uuid + + "][" + + clusterUUID + + "]"; + assert indices.values().containsAll(shardGenerations.indices()) + : "ShardGenerations contained indices " + + shardGenerations.indices() + + " but snapshots only reference indices " + + indices.values(); + assert indexSnapshots.values().stream().noneMatch(snapshotIdList -> Set.copyOf(snapshotIdList).size() != snapshotIdList.size()) + : "Found duplicate snapshot ids per index in [" + indexSnapshots + "]"; } protected RepositoryData copy() { return new RepositoryData( - uuid, - genId, - snapshotIds, - snapshotsDetails, - indices, - indexSnapshots, - shardGenerations, - indexMetaDataGenerations, - clusterUUID); + uuid, + genId, + snapshotIds, + snapshotsDetails, + indices, + indexSnapshots, + shardGenerations, + indexMetaDataGenerations, + clusterUUID + ); } /** @@ -191,15 +204,16 @@ protected RepositoryData copy() { */ public RepositoryData withoutShardGenerations() { return new RepositoryData( - uuid, - genId, - snapshotIds, - snapshotsDetails, - indices, - indexSnapshots, - ShardGenerations.EMPTY, - indexMetaDataGenerations, - clusterUUID); + uuid, + genId, + snapshotIds, + snapshotsDetails, + indices, + indexSnapshots, + ShardGenerations.EMPTY, + indexMetaDataGenerations, + clusterUUID + ); } /** @@ -214,15 +228,16 @@ public RepositoryData withExtraDetails(Map extraDet final Map newDetails = new HashMap<>(snapshotsDetails); extraDetails.forEach((id, extraDetail) -> newDetails.put(id.getUUID(), extraDetail)); return new RepositoryData( - uuid, - genId, - snapshotIds, - newDetails, - indices, - indexSnapshots, - shardGenerations, - indexMetaDataGenerations, - clusterUUID); + uuid, + genId, + snapshotIds, + newDetails, + indices, + indexSnapshots, + shardGenerations, + indexMetaDataGenerations, + clusterUUID + ); } public ShardGenerations shardGenerations() { @@ -266,9 +281,9 @@ public Collection getSnapshotIds() { public boolean hasMissingDetails(SnapshotId snapshotId) { final SnapshotDetails snapshotDetails = getSnapshotDetails(snapshotId); return snapshotDetails == null - || snapshotDetails.getVersion() == null - || snapshotDetails.getStartTimeMillis() == -1 - || snapshotDetails.getEndTimeMillis() == -1; + || snapshotDetails.getVersion() == null + || snapshotDetails.getStartTimeMillis() == -1 + || snapshotDetails.getEndTimeMillis() == -1; } /** @@ -311,19 +326,18 @@ public Map getIndices() { * @return List of indices that are changed but not removed */ public List indicesToUpdateAfterRemovingSnapshot(Collection snapshotIds) { - return indexSnapshots.entrySet().stream() - .filter(entry -> { - final Collection existingIds = entry.getValue(); - if (snapshotIds.containsAll(existingIds)) { - return existingIds.size() > snapshotIds.size(); - } - for (SnapshotId snapshotId : snapshotIds) { - if (entry.getValue().contains(snapshotId)) { - return true; - } + return indexSnapshots.entrySet().stream().filter(entry -> { + final Collection existingIds = entry.getValue(); + if (snapshotIds.containsAll(existingIds)) { + return existingIds.size() > snapshotIds.size(); + } + for (SnapshotId snapshotId : snapshotIds) { + if (entry.getValue().contains(snapshotId)) { + return true; } - return false; - }).map(Map.Entry::getKey).collect(Collectors.toList()); + } + return false; + }).map(Map.Entry::getKey).collect(Collectors.toList()); } /** @@ -336,9 +350,12 @@ public List indicesToUpdateAfterRemovingSnapshot(Collection */ public Map> indexMetaDataToRemoveAfterRemovingSnapshots(Collection snapshotIds) { Collection indicesForSnapshot = indicesToUpdateAfterRemovingSnapshot(snapshotIds); - final Set allRemainingIdentifiers = indexMetaDataGenerations.lookup.entrySet().stream() - .filter(e -> snapshotIds.contains(e.getKey()) == false).flatMap(e -> e.getValue().values().stream()) - .map(indexMetaDataGenerations::getIndexMetaBlobId).collect(Collectors.toSet()); + final Set allRemainingIdentifiers = indexMetaDataGenerations.lookup.entrySet() + .stream() + .filter(e -> snapshotIds.contains(e.getKey()) == false) + .flatMap(e -> e.getValue().values().stream()) + .map(indexMetaDataGenerations::getIndexMetaBlobId) + .collect(Collectors.toSet()); final Map> toRemove = new HashMap<>(); for (IndexId indexId : indicesForSnapshot) { for (SnapshotId snapshotId : snapshotIds) { @@ -363,11 +380,13 @@ public Map> indexMetaDataToRemoveAfterRemovingSnapsh * @param newIdentifiers Map of new index metadata blob uuids keyed by the identifiers of the * {@link IndexMetadata} in them */ - public RepositoryData addSnapshot(final SnapshotId snapshotId, - final SnapshotDetails details, - final ShardGenerations shardGenerations, - @Nullable final Map indexMetaBlobs, - @Nullable final Map newIdentifiers) { + public RepositoryData addSnapshot( + final SnapshotId snapshotId, + final SnapshotDetails details, + final ShardGenerations shardGenerations, + @Nullable final Map indexMetaBlobs, + @Nullable final Map newIdentifiers + ) { if (snapshotIds.containsKey(snapshotId.getUUID())) { // if the snapshot id already exists in the repository data, it means an old master // that is blocked from the cluster is trying to finalize a snapshot concurrently with @@ -391,25 +410,28 @@ public RepositoryData addSnapshot(final SnapshotId snapshotId, final IndexMetaDataGenerations newIndexMetaGenerations; if (indexMetaBlobs == null) { assert newIdentifiers == null : "Non-null new identifiers [" + newIdentifiers + "] for null lookup"; - assert indexMetaDataGenerations.lookup.isEmpty() : - "Index meta generations should have been empty but was [" + indexMetaDataGenerations + "]"; + assert indexMetaDataGenerations.lookup.isEmpty() + : "Index meta generations should have been empty but was [" + indexMetaDataGenerations + "]"; newIndexMetaGenerations = IndexMetaDataGenerations.EMPTY; } else { - assert indexMetaBlobs.isEmpty() || shardGenerations.indices().equals(indexMetaBlobs.keySet()) : - "Shard generations contained indices " + shardGenerations.indices() - + " but indexMetaData was given for " + indexMetaBlobs.keySet(); + assert indexMetaBlobs.isEmpty() || shardGenerations.indices().equals(indexMetaBlobs.keySet()) + : "Shard generations contained indices " + + shardGenerations.indices() + + " but indexMetaData was given for " + + indexMetaBlobs.keySet(); newIndexMetaGenerations = indexMetaDataGenerations.withAddedSnapshot(snapshotId, indexMetaBlobs, newIdentifiers); } return new RepositoryData( - uuid, - genId, - snapshots, - newSnapshotDetails, - allIndexSnapshots, - ShardGenerations.builder().putAll(this.shardGenerations).putAll(shardGenerations).build(), - newIndexMetaGenerations, - clusterUUID); + uuid, + genId, + snapshots, + newSnapshotDetails, + allIndexSnapshots, + ShardGenerations.builder().putAll(this.shardGenerations).putAll(shardGenerations).build(), + newIndexMetaGenerations, + clusterUUID + ); } /** @@ -423,15 +445,16 @@ public RepositoryData withGenId(long newGeneration) { return this; } return new RepositoryData( - uuid, - newGeneration, - snapshotIds, - snapshotsDetails, - indices, - indexSnapshots, - shardGenerations, - indexMetaDataGenerations, - clusterUUID); + uuid, + newGeneration, + snapshotIds, + snapshotsDetails, + indices, + indexSnapshots, + shardGenerations, + indexMetaDataGenerations, + clusterUUID + ); } /** @@ -440,29 +463,31 @@ public RepositoryData withGenId(long newGeneration) { */ public RepositoryData withoutUUIDs() { return new RepositoryData( - MISSING_UUID, - genId, - snapshotIds, - snapshotsDetails, - indices, - indexSnapshots, - shardGenerations, - indexMetaDataGenerations, - MISSING_UUID); + MISSING_UUID, + genId, + snapshotIds, + snapshotsDetails, + indices, + indexSnapshots, + shardGenerations, + indexMetaDataGenerations, + MISSING_UUID + ); } public RepositoryData withClusterUuid(String clusterUUID) { assert clusterUUID.equals(MISSING_UUID) == false; return new RepositoryData( - uuid.equals(MISSING_UUID) ? UUIDs.randomBase64UUID() : uuid, - genId, - snapshotIds, - snapshotsDetails, - indices, - indexSnapshots, - shardGenerations, - indexMetaDataGenerations, - clusterUUID); + uuid.equals(MISSING_UUID) ? UUIDs.randomBase64UUID() : uuid, + genId, + snapshotIds, + snapshotsDetails, + indices, + indexSnapshots, + shardGenerations, + indexMetaDataGenerations, + clusterUUID + ); } /** @@ -474,7 +499,9 @@ public RepositoryData withClusterUuid(String clusterUUID) { * changed shard indexed by its shardId */ public RepositoryData removeSnapshots(final Collection snapshots, final ShardGenerations updatedShardGenerations) { - Map newSnapshotIds = snapshotIds.values().stream().filter(Predicate.not(snapshots::contains)) + Map newSnapshotIds = snapshotIds.values() + .stream() + .filter(Predicate.not(snapshots::contains)) .collect(Collectors.toMap(SnapshotId::getUUID, Function.identity())); if (newSnapshotIds.size() != snapshotIds.size() - snapshots.size()) { final Collection notFound = new HashSet<>(snapshots); @@ -501,15 +528,18 @@ public RepositoryData removeSnapshots(final Collection snapshots, fi } return new RepositoryData( - uuid, - genId, - newSnapshotIds, - newSnapshotsDetails, - indexSnapshots, - ShardGenerations.builder().putAll(shardGenerations).putAll(updatedShardGenerations) - .retainIndicesAndPruneDeletes(indexSnapshots.keySet()).build(), - indexMetaDataGenerations.withRemovedSnapshots(snapshots), - clusterUUID + uuid, + genId, + newSnapshotIds, + newSnapshotsDetails, + indexSnapshots, + ShardGenerations.builder() + .putAll(shardGenerations) + .putAll(updatedShardGenerations) + .retainIndicesAndPruneDeletes(indexSnapshots.keySet()) + .build(), + indexMetaDataGenerations.withRemovedSnapshots(snapshots), + clusterUUID ); } @@ -534,17 +564,16 @@ public boolean equals(Object obj) { } RepositoryData that = (RepositoryData) obj; return snapshotIds.equals(that.snapshotIds) - && snapshotsDetails.equals(that.snapshotsDetails) - && indices.equals(that.indices) - && indexSnapshots.equals(that.indexSnapshots) - && shardGenerations.equals(that.shardGenerations) - && indexMetaDataGenerations.equals(that.indexMetaDataGenerations); + && snapshotsDetails.equals(that.snapshotsDetails) + && indices.equals(that.indices) + && indexSnapshots.equals(that.indexSnapshots) + && shardGenerations.equals(that.shardGenerations) + && indexMetaDataGenerations.equals(that.indexMetaDataGenerations); } @Override public int hashCode() { - return Objects.hash( - snapshotIds, snapshotsDetails, indices, indexSnapshots, shardGenerations, indexMetaDataGenerations); + return Objects.hash(snapshotIds, snapshotsDetails, indices, indexSnapshots, shardGenerations, indexMetaDataGenerations); } /** @@ -616,10 +645,8 @@ public XContentBuilder snapshotsToXContent(final XContentBuilder builder, final * @param permitMissingUuid indicates whether we permit the repository- and cluster UUIDs to be missing, * e.g. we are serializing for the in-memory cache or running tests */ - public XContentBuilder snapshotsToXContent( - final XContentBuilder builder, - final Version repoMetaVersion, - boolean permitMissingUuid) throws IOException { + public XContentBuilder snapshotsToXContent(final XContentBuilder builder, final Version repoMetaVersion, boolean permitMissingUuid) + throws IOException { final boolean shouldWriteUUIDS = SnapshotsService.includesUUIDs(repoMetaVersion); final boolean shouldWriteIndexGens = SnapshotsService.useIndexGenerations(repoMetaVersion); @@ -674,8 +701,8 @@ public XContentBuilder snapshotsToXContent( } if (shouldWriteIndexGens) { builder.startObject(INDEX_METADATA_LOOKUP); - for (Map.Entry entry : indexMetaDataGenerations.lookup.getOrDefault( - snapshot, Collections.emptyMap()).entrySet()) { + for (Map.Entry entry : indexMetaDataGenerations.lookup.getOrDefault(snapshot, Collections.emptyMap()) + .entrySet()) { builder.field(entry.getKey().getId(), entry.getValue()); } builder.endObject(); @@ -770,7 +797,8 @@ public static RepositoryData snapshotsFromXContent(XContentParser parser, long g assert SnapshotsService.useShardGenerations(version); if (version.after(Version.CURRENT)) { throw new IllegalStateException( - "this snapshot repository format requires Elasticsearch version [" + version + "] or later"); + "this snapshot repository format requires Elasticsearch version [" + version + "] or later" + ); } break; case UUID: @@ -792,14 +820,15 @@ public static RepositoryData snapshotsFromXContent(XContentParser parser, long g XContentParserUtils.ensureExpectedToken(null, parser.nextToken(), parser); return new RepositoryData( - uuid, - genId, - snapshots, - snapshotsDetails, - indexSnapshots, - shardGenerations.build(), - buildIndexMetaGenerations(indexMetaLookup, indexLookup, indexMetaIdentifiers), - clusterUUID); + uuid, + genId, + snapshots, + snapshotsDetails, + indexSnapshots, + shardGenerations.build(), + buildIndexMetaGenerations(indexMetaLookup, indexLookup, indexMetaIdentifiers), + clusterUUID + ); } /** @@ -812,9 +841,11 @@ public static RepositoryData snapshotsFromXContent(XContentParser parser, long g * @param indexMetaIdentifiers map of index generation to index meta identifiers parsed by {@link #snapshotsFromXContent} * @return index meta generations instance */ - private static IndexMetaDataGenerations buildIndexMetaGenerations(Map> indexMetaLookup, - Map indexLookup, - Map indexMetaIdentifiers) { + private static IndexMetaDataGenerations buildIndexMetaGenerations( + Map> indexMetaLookup, + Map indexLookup, + Map indexMetaIdentifiers + ) { if (indexMetaLookup.isEmpty()) { return IndexMetaDataGenerations.EMPTY; } @@ -842,10 +873,11 @@ private static IndexMetaDataGenerations buildIndexMetaGenerations(Map snapshots, - Map snapshotsDetails, - Map> indexMetaLookup) throws IOException { + XContentParser parser, + Map snapshots, + Map snapshotsDetails, + Map> indexMetaLookup + ) throws IOException { XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.nextToken(), parser); final Map stringDeduplicator = new HashMap<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { @@ -885,8 +917,7 @@ private static void parseSnapshots( break; } } - assert (startTimeMillis == -1) == (endTimeMillis == -1) - : "unexpected: " + startTimeMillis + ", " + endTimeMillis + ", "; + assert (startTimeMillis == -1) == (endTimeMillis == -1) : "unexpected: " + startTimeMillis + ", " + endTimeMillis + ", "; final SnapshotId snapshotId = new SnapshotId(name, uuid); if (state != null || version != null) { snapshotsDetails.put(uuid, new SnapshotDetails(state, version, startTimeMillis, endTimeMillis)); @@ -909,9 +940,14 @@ private static void parseSnapshots( * @param indexLookup map of index uuid (as returned by {@link IndexId#getId}) to {@link IndexId} * @param shardGenerations shard generations builder that is populated index by this method */ - private static void parseIndices(XContentParser parser, boolean fixBrokenShardGens, Map snapshots, - Map> indexSnapshots, Map indexLookup, - ShardGenerations.Builder shardGenerations) throws IOException { + private static void parseIndices( + XContentParser parser, + boolean fixBrokenShardGens, + Map snapshots, + Map> indexSnapshots, + Map indexLookup, + ShardGenerations.Builder shardGenerations + ) throws IOException { XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); while (parser.nextToken() != XContentParser.Token.END_OBJECT) { final String indexName = parser.currentName(); @@ -946,8 +982,13 @@ private static void parseIndices(XContentParser parser, boolean fixBrokenShardGe // A snapshotted index references a snapshot which does not exist in // the list of snapshots. This can happen when multiple clusters in // different versions create or delete snapshot in the same repository. - throw new ElasticsearchParseException("Detected a corrupted repository, index " + indexId - + " references an unknown snapshot uuid [" + uuid + "]"); + throw new ElasticsearchParseException( + "Detected a corrupted repository, index " + + indexId + + " references an unknown snapshot uuid [" + + uuid + + "]" + ); } snapshotIds.add(snapshotId); } @@ -1007,11 +1048,7 @@ public static class SnapshotDetails { // May be -1 if unknown, which happens if the snapshot was taken before 7.14 and hasn't been updated yet private final long endTimeMillis; - public SnapshotDetails( - @Nullable SnapshotState snapshotState, - @Nullable Version version, - long startTimeMillis, - long endTimeMillis) { + public SnapshotDetails(@Nullable SnapshotState snapshotState, @Nullable Version version, long startTimeMillis, long endTimeMillis) { this.snapshotState = snapshotState; this.version = version; this.startTimeMillis = startTimeMillis; @@ -1048,9 +1085,9 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; SnapshotDetails that = (SnapshotDetails) o; return startTimeMillis == that.startTimeMillis - && endTimeMillis == that.endTimeMillis - && snapshotState == that.snapshotState - && Objects.equals(version, that.version); + && endTimeMillis == that.endTimeMillis + && snapshotState == that.snapshotState + && Objects.equals(version, that.version); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryException.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryException.java index 3cae91b59f9a7..c76bed5db3d6c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryException.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryException.java @@ -38,7 +38,7 @@ public String repository() { return repository; } - public RepositoryException(StreamInput in) throws IOException{ + public RepositoryException(StreamInput in) throws IOException { super(in); repository = in.readOptionalString(); } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryInfo.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryInfo.java index 5d59f3be804a4..e2dfe48c52b75 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryInfo.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryInfo.java @@ -29,20 +29,18 @@ public final class RepositoryInfo implements Writeable, ToXContentFragment { @Nullable public final Long stoppedAt; - public RepositoryInfo(String ephemeralId, - String name, - String type, - Map location, - long startedAt) { + public RepositoryInfo(String ephemeralId, String name, String type, Map location, long startedAt) { this(ephemeralId, name, type, location, startedAt, null); } - public RepositoryInfo(String ephemeralId, - String name, - String type, - Map location, - long startedAt, - @Nullable Long stoppedAt) { + public RepositoryInfo( + String ephemeralId, + String name, + String type, + Map location, + long startedAt, + @Nullable Long stoppedAt + ) { this.ephemeralId = ephemeralId; this.name = name; this.type = type; @@ -101,12 +99,12 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; RepositoryInfo that = (RepositoryInfo) o; - return ephemeralId.equals(that.ephemeralId) && - name.equals(that.name) && - type.equals(that.type) && - location.equals(that.location) && - startedAt == that.startedAt && - Objects.equals(stoppedAt, that.stoppedAt); + return ephemeralId.equals(that.ephemeralId) + && name.equals(that.name) + && type.equals(that.type) + && location.equals(that.location) + && startedAt == that.startedAt + && Objects.equals(stoppedAt, that.stoppedAt); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryMissingException.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryMissingException.java index 2302fb4eecd7b..3ae4000463ed8 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryMissingException.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryMissingException.java @@ -18,7 +18,6 @@ */ public class RepositoryMissingException extends RepositoryException { - public RepositoryMissingException(String repository) { super(repository, "missing"); } @@ -28,7 +27,7 @@ public RestStatus status() { return RestStatus.NOT_FOUND; } - public RepositoryMissingException(StreamInput in) throws IOException{ + public RepositoryMissingException(StreamInput in) throws IOException { super(in); } } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryStats.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryStats.java index 9169a4e39adc4..aee9d7c7cced2 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryStats.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryStats.java @@ -61,8 +61,6 @@ public int hashCode() { @Override public String toString() { - return "RepositoryStats{" + - "requestCounts=" + requestCounts + - '}'; + return "RepositoryStats{" + "requestCounts=" + requestCounts + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryStatsSnapshot.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryStatsSnapshot.java index 4aa859fce2ae4..e352a9e282161 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryStatsSnapshot.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryStatsSnapshot.java @@ -25,10 +25,7 @@ public final class RepositoryStatsSnapshot implements Writeable, ToXContentObjec private final long clusterVersion; private final boolean archived; - public RepositoryStatsSnapshot(RepositoryInfo repositoryInfo, - RepositoryStats repositoryStats, - long clusterVersion, - boolean archived) { + public RepositoryStatsSnapshot(RepositoryInfo repositoryInfo, RepositoryStats repositoryStats, long clusterVersion, boolean archived) { assert archived != (clusterVersion == UNKNOWN_CLUSTER_VERSION); this.repositoryInfo = repositoryInfo; this.repositoryStats = repositoryStats; @@ -85,10 +82,10 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; RepositoryStatsSnapshot that = (RepositoryStatsSnapshot) o; - return repositoryInfo.equals(that.repositoryInfo) && - repositoryStats.equals(that.repositoryStats) && - clusterVersion == that.clusterVersion && - archived == that.archived; + return repositoryInfo.equals(that.repositoryInfo) + && repositoryStats.equals(that.repositoryStats) + && clusterVersion == that.clusterVersion + && archived == that.archived; } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryVerificationException.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryVerificationException.java index e68bca87bfd52..a5353e2cf0019 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryVerificationException.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryVerificationException.java @@ -18,7 +18,6 @@ */ public class RepositoryVerificationException extends RepositoryException { - public RepositoryVerificationException(String repository, String msg) { super(repository, msg); } @@ -32,7 +31,7 @@ public RestStatus status() { return RestStatus.INTERNAL_SERVER_ERROR; } - public RepositoryVerificationException(StreamInput in) throws IOException{ + public RepositoryVerificationException(StreamInput in) throws IOException { super(in); } @@ -42,4 +41,3 @@ public synchronized Throwable fillInStackTrace() { return this; } } - diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardGenerations.java b/server/src/main/java/org/elasticsearch/repositories/ShardGenerations.java index 897973705935f..0b1dd9831a044 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardGenerations.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardGenerations.java @@ -208,25 +208,22 @@ public Builder putAll(ShardGenerations shardGenerations) { public Builder put(IndexId indexId, int shardId, String generation) { String existingGeneration = generations.computeIfAbsent(indexId, i -> new HashMap<>()).put(shardId, generation); - assert generation != null || existingGeneration == null : - "must not overwrite existing generation with null generation [" + existingGeneration + "]"; + assert generation != null || existingGeneration == null + : "must not overwrite existing generation with null generation [" + existingGeneration + "]"; return this; } public ShardGenerations build() { - return new ShardGenerations(generations.entrySet().stream().collect(Collectors.toMap( - Map.Entry::getKey, - entry -> { - final Set shardIds = entry.getValue().keySet(); - assert shardIds.isEmpty() == false; - final int size = shardIds.stream().mapToInt(i -> i).max().getAsInt() + 1; - // Create a list that can hold the highest shard id as index and leave null values for shards that don't have - // a map entry. - final String[] gens = new String[size]; - entry.getValue().forEach((shardId, generation) -> gens[shardId] = generation); - return Collections.unmodifiableList(Arrays.asList(gens)); - } - ))); + return new ShardGenerations(generations.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> { + final Set shardIds = entry.getValue().keySet(); + assert shardIds.isEmpty() == false; + final int size = shardIds.stream().mapToInt(i -> i).max().getAsInt() + 1; + // Create a list that can hold the highest shard id as index and leave null values for shards that don't have + // a map entry. + final String[] gens = new String[size]; + entry.getValue().forEach((shardId, generation) -> gens[shardId] = generation); + return Collections.unmodifiableList(Arrays.asList(gens)); + }))); } } } diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotResult.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotResult.java index bdeae8ebc000c..2096712f69973 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotResult.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotResult.java @@ -89,10 +89,6 @@ public int hashCode() { @Override public String toString() { - return "ShardSnapshotResult{" + - "generation='" + generation + '\'' + - ", size=" + size + - ", segmentCount=" + segmentCount + - '}'; + return "ShardSnapshotResult{" + "generation='" + generation + '\'' + ", size=" + size + ", segmentCount=" + segmentCount + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java index e8d2186291b23..c9c094c7097a7 100644 --- a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java @@ -54,16 +54,17 @@ public final class SnapshotShardContext extends ActionListener.Delegating userMetadata, - ActionListener listener) { + Store store, + MapperService mapperService, + SnapshotId snapshotId, + IndexId indexId, + Engine.IndexCommitRef commitRef, + @Nullable String shardStateIdentifier, + IndexShardSnapshotStatus snapshotStatus, + Version repositoryMetaVersion, + Map userMetadata, + ActionListener listener + ) { super(ActionListener.runBefore(listener, commitRef::close)); this.store = store; this.mapperService = mapperService; diff --git a/server/src/main/java/org/elasticsearch/repositories/VerifyNodeRepositoryAction.java b/server/src/main/java/org/elasticsearch/repositories/VerifyNodeRepositoryAction.java index 23adf36f31321..ce278e2fdf259 100644 --- a/server/src/main/java/org/elasticsearch/repositories/VerifyNodeRepositoryAction.java +++ b/server/src/main/java/org/elasticsearch/repositories/VerifyNodeRepositoryAction.java @@ -47,13 +47,20 @@ public class VerifyNodeRepositoryAction { private final RepositoriesService repositoriesService; - public VerifyNodeRepositoryAction(TransportService transportService, ClusterService clusterService, - RepositoriesService repositoriesService) { + public VerifyNodeRepositoryAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService + ) { this.transportService = transportService; this.clusterService = clusterService; this.repositoriesService = repositoriesService; - transportService.registerRequestHandler(ACTION_NAME, ThreadPool.Names.SNAPSHOT, VerifyNodeRepositoryRequest::new, - new VerifyNodeRepositoryRequestHandler()); + transportService.registerRequestHandler( + ACTION_NAME, + ThreadPool.Names.SNAPSHOT, + VerifyNodeRepositoryRequest::new, + new VerifyNodeRepositoryRequestHandler() + ); } public void verify(String repository, String verificationToken, final ActionListener> listener) { @@ -82,7 +89,10 @@ public void verify(String repository, String verificationToken, final ActionList finishVerification(repository, listener, nodes, errors); } } else { - transportService.sendRequest(node, ACTION_NAME, new VerifyNodeRepositoryRequest(repository, verificationToken), + transportService.sendRequest( + node, + ACTION_NAME, + new VerifyNodeRepositoryRequest(repository, verificationToken), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { @Override public void handleResponse(TransportResponse.Empty response) { @@ -98,13 +108,18 @@ public void handleException(TransportException exp) { finishVerification(repository, listener, nodes, errors); } } - }); + } + ); } } } - private static void finishVerification(String repositoryName, ActionListener> listener, List nodes, - CopyOnWriteArrayList errors) { + private static void finishVerification( + String repositoryName, + ActionListener> listener, + List nodes, + CopyOnWriteArrayList errors + ) { if (errors.isEmpty() == false) { RepositoryVerificationException e = new RepositoryVerificationException(repositoryName, errors.toString()); for (VerificationFailure error : errors) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 0615f60337b0f..aff5efbad19b9 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -208,15 +208,22 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp /** * Setting to disable caching of the latest repository data. */ - public static final Setting CACHE_REPOSITORY_DATA = - Setting.boolSetting("cache_repository_data", true, Setting.Property.Deprecated); + public static final Setting CACHE_REPOSITORY_DATA = Setting.boolSetting( + "cache_repository_data", + true, + Setting.Property.Deprecated + ); /** * Size hint for the IO buffer size to use when reading from and writing to the repository. */ - public static final Setting BUFFER_SIZE_SETTING = Setting.byteSizeSetting("io_buffer_size", - ByteSizeValue.parseBytesSizeValue("128kb", "io_buffer_size"), ByteSizeValue.parseBytesSizeValue("8kb", "buffer_size"), - ByteSizeValue.parseBytesSizeValue("16mb", "io_buffer_size"), Setting.Property.NodeScope); + public static final Setting BUFFER_SIZE_SETTING = Setting.byteSizeSetting( + "io_buffer_size", + ByteSizeValue.parseBytesSizeValue("128kb", "io_buffer_size"), + ByteSizeValue.parseBytesSizeValue("8kb", "buffer_size"), + ByteSizeValue.parseBytesSizeValue("16mb", "io_buffer_size"), + Setting.Property.NodeScope + ); /** * Setting to disable writing the {@code index.latest} blob which enables the contents of this repository to be used with a @@ -228,8 +235,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp * Setting that defines the maximum number of snapshots to which the repository may grow. Trying to create a snapshot into the * repository that would move it above this size will throw an exception. */ - public static final Setting MAX_SNAPSHOTS_SETTING = - Setting.intSetting("max_number_of_snapshots", 500, 1, Setting.Property.NodeScope); + public static final Setting MAX_SNAPSHOTS_SETTING = Setting.intSetting( + "max_number_of_snapshots", + 500, + 1, + Setting.Property.NodeScope + ); protected final boolean supportURLRepo; @@ -245,34 +256,59 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private final CounterMetric restoreRateLimitingTimeInNanos = new CounterMetric(); - public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = - new ChecksumBlobStoreFormat<>("metadata", METADATA_NAME_FORMAT, Metadata::fromXContent); + public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "metadata", + METADATA_NAME_FORMAT, + Metadata::fromXContent + ); - public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = - new ChecksumBlobStoreFormat<>("index-metadata", METADATA_NAME_FORMAT, IndexMetadata::fromXContent); + public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "index-metadata", + METADATA_NAME_FORMAT, + IndexMetadata::fromXContent + ); private static final String SNAPSHOT_CODEC = "snapshot"; - public static final ChecksumBlobStoreFormat SNAPSHOT_FORMAT = - new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, SnapshotInfo::fromXContentInternal); - - public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOT_FORMAT = - new ChecksumBlobStoreFormat<>(SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, BlobStoreIndexShardSnapshot::fromXContent); - - public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOTS_FORMAT = - new ChecksumBlobStoreFormat<>("snapshots", SNAPSHOT_INDEX_NAME_FORMAT, BlobStoreIndexShardSnapshots::fromXContent); - - public static final Setting MAX_SNAPSHOT_BYTES_PER_SEC = Setting.byteSizeSetting("max_snapshot_bytes_per_sec", - new ByteSizeValue(40, ByteSizeUnit.MB), Setting.Property.Dynamic, Setting.Property.NodeScope); - - public static final Setting MAX_RESTORE_BYTES_PER_SEC = Setting.byteSizeSetting("max_restore_bytes_per_sec", - ByteSizeValue.ZERO, Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final ChecksumBlobStoreFormat SNAPSHOT_FORMAT = new ChecksumBlobStoreFormat<>( + SNAPSHOT_CODEC, + SNAPSHOT_NAME_FORMAT, + SnapshotInfo::fromXContentInternal + ); + + public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOT_FORMAT = new ChecksumBlobStoreFormat<>( + SNAPSHOT_CODEC, + SNAPSHOT_NAME_FORMAT, + BlobStoreIndexShardSnapshot::fromXContent + ); + + public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOTS_FORMAT = new ChecksumBlobStoreFormat<>( + "snapshots", + SNAPSHOT_INDEX_NAME_FORMAT, + BlobStoreIndexShardSnapshots::fromXContent + ); + + public static final Setting MAX_SNAPSHOT_BYTES_PER_SEC = Setting.byteSizeSetting( + "max_snapshot_bytes_per_sec", + new ByteSizeValue(40, ByteSizeUnit.MB), + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + public static final Setting MAX_RESTORE_BYTES_PER_SEC = Setting.byteSizeSetting( + "max_restore_bytes_per_sec", + ByteSizeValue.ZERO, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); /** * Repository settings that can be updated dynamically without having to create a new repository. */ - private static final Set DYNAMIC_SETTING_NAMES = - Set.of(MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), MAX_RESTORE_BYTES_PER_SEC.getKey()); + private static final Set DYNAMIC_SETTING_NAMES = Set.of( + MAX_SNAPSHOT_BYTES_PER_SEC.getKey(), + MAX_RESTORE_BYTES_PER_SEC.getKey() + ); private final boolean readOnly; @@ -338,7 +374,8 @@ protected BlobStoreRepository( final ClusterService clusterService, final BigArrays bigArrays, final RecoverySettings recoverySettings, - final BlobPath basePath) { + final BlobPath basePath + ) { this.metadata = metadata; this.threadPool = clusterService.getClusterApplierService().threadPool(); this.clusterService = clusterService; @@ -358,8 +395,8 @@ protected BlobStoreRepository( @Override protected void doStart() { - uncleanStart = metadata.pendingGeneration() > RepositoryData.EMPTY_REPO_GEN && - metadata.generation() != metadata.pendingGeneration(); + uncleanStart = metadata.pendingGeneration() > RepositoryData.EMPTY_REPO_GEN + && metadata.generation() != metadata.pendingGeneration(); ByteSizeValue chunkSize = chunkSize(); if (chunkSize != null && chunkSize.getBytes() <= 0) { throw new IllegalArgumentException("the chunk size cannot be negative: [" + chunkSize + "]"); @@ -367,8 +404,7 @@ protected void doStart() { } @Override - protected void doStop() { - } + protected void doStop() {} @Override protected void doClose() { @@ -387,8 +423,11 @@ protected void doClose() { } @Override - public void executeConsistentStateUpdate(Function createUpdateTask, String source, - Consumer onFailure) { + public void executeConsistentStateUpdate( + Function createUpdateTask, + String source, + Consumer onFailure + ) { final RepositoryMetadata repositoryMetadataStart = metadata; getRepositoryData(ActionListener.wrap(repositoryData -> { final ClusterStateUpdateTask updateTask = createUpdateTask.apply(repositoryData); @@ -433,8 +472,13 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS } @Override - public void cloneShardSnapshot(SnapshotId source, SnapshotId target, RepositoryShardId shardId, - @Nullable String shardGeneration, ActionListener listener) { + public void cloneShardSnapshot( + SnapshotId source, + SnapshotId target, + RepositoryShardId shardId, + @Nullable String shardGeneration, + ActionListener listener + ) { if (isReadOnly()) { listener.onFailure(new RepositoryException(metadata.name(), "cannot clone shard snapshot on a readonly repository")); return; @@ -450,7 +494,9 @@ public void cloneShardSnapshot(SnapshotId source, SnapshotId target, RepositoryS final String existingShardGen; if (shardGeneration == null) { Tuple tuple = buildBlobStoreIndexShardSnapshots( - shardContainer.listBlobsByPrefix(INDEX_FILE_PREFIX).keySet(), shardContainer); + shardContainer.listBlobsByPrefix(INDEX_FILE_PREFIX).keySet(), + shardContainer + ); existingShardGen = String.valueOf(tuple.v2()); newGen = String.valueOf(tuple.v2() + 1); existingSnapshots = tuple.v1(); @@ -473,35 +519,60 @@ public void cloneShardSnapshot(SnapshotId source, SnapshotId target, RepositoryS } } if (sourceFiles == null) { - throw new RepositoryException(metadata.name(), "Can't create clone of [" + shardId + "] for snapshot [" - + target + "]. The source snapshot [" + source + "] was not found in the shard metadata."); + throw new RepositoryException( + metadata.name(), + "Can't create clone of [" + + shardId + + "] for snapshot [" + + target + + "]. The source snapshot [" + + source + + "] was not found in the shard metadata." + ); } if (existingTargetFiles != null) { if (existingTargetFiles.isSame(sourceFiles)) { return new ShardSnapshotResult( - existingShardGen, - ByteSizeValue.ofBytes(existingTargetFiles.totalSize()), - getSegmentInfoFileCount(existingTargetFiles.indexFiles())); + existingShardGen, + ByteSizeValue.ofBytes(existingTargetFiles.totalSize()), + getSegmentInfoFileCount(existingTargetFiles.indexFiles()) + ); } - throw new RepositoryException(metadata.name(), "Can't create clone of [" + shardId + "] for snapshot [" - + target + "]. A snapshot by that name already exists for this shard."); + throw new RepositoryException( + metadata.name(), + "Can't create clone of [" + + shardId + + "] for snapshot [" + + target + + "]. A snapshot by that name already exists for this shard." + ); } final BlobStoreIndexShardSnapshot sourceMeta = loadShardSnapshot(shardContainer, source); logger.trace("[{}] [{}] writing shard snapshot file for clone", shardId, target); - INDEX_SHARD_SNAPSHOT_FORMAT.write(sourceMeta.asClone(target.getName(), startTime, - threadPool.absoluteTimeInMillis() - startTime), - shardContainer, target.getUUID(), compress, bigArrays); - INDEX_SHARD_SNAPSHOTS_FORMAT.write(existingSnapshots.withClone(source.getName(), target.getName()), shardContainer, newGen, - compress, bigArrays); + INDEX_SHARD_SNAPSHOT_FORMAT.write( + sourceMeta.asClone(target.getName(), startTime, threadPool.absoluteTimeInMillis() - startTime), + shardContainer, + target.getUUID(), + compress, + bigArrays + ); + INDEX_SHARD_SNAPSHOTS_FORMAT.write( + existingSnapshots.withClone(source.getName(), target.getName()), + shardContainer, + newGen, + compress, + bigArrays + ); return new ShardSnapshotResult( - newGen, - ByteSizeValue.ofBytes(sourceMeta.totalSize()), - getSegmentInfoFileCount(sourceMeta.indexFiles())); + newGen, + ByteSizeValue.ofBytes(sourceMeta.totalSize()), + getSegmentInfoFileCount(sourceMeta.indexFiles()) + ); })); } private static int getSegmentInfoFileCount(List indexFiles) { - //noinspection ConstantConditions + // noinspection ConstantConditions return Math.toIntExact(Math.min(Integer.MAX_VALUE, indexFiles.stream().filter(fi -> fi.physicalName().endsWith(".si")).count())); } @@ -545,21 +616,24 @@ public void updateState(ClusterState state) { // In this case, the generation points at the generation the repo will be in after the snapshot finishes so it may not yet // exist if (bestGenerationFromCS == RepositoryData.EMPTY_REPO_GEN) { - bestGenerationFromCS = - bestGeneration(state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()); + bestGenerationFromCS = bestGeneration( + state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries() + ); } if (bestGenerationFromCS == RepositoryData.EMPTY_REPO_GEN) { - bestGenerationFromCS = - bestGeneration(state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).entries()); + bestGenerationFromCS = bestGeneration( + state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).entries() + ); } final long finalBestGen = Math.max(bestGenerationFromCS, metadata.generation()); latestKnownRepoGen.updateAndGet(known -> Math.max(known, finalBestGen)); } else { final long previousBest = latestKnownRepoGen.getAndSet(metadata.generation()); if (previousBest != metadata.generation()) { - assert wasBestEffortConsistency || metadata.generation() == RepositoryData.CORRUPTED_REPO_GEN - || previousBest < metadata.generation() : "Illegal move from repository generation [" + previousBest - + "] to generation [" + metadata.generation() + "]"; + assert wasBestEffortConsistency + || metadata.generation() == RepositoryData.CORRUPTED_REPO_GEN + || previousBest < metadata.generation() + : "Illegal move from repository generation [" + previousBest + "] to generation [" + metadata.generation() + "]"; logger.debug("Updated repository generation from [{}] to [{}]", previousBest, metadata.generation()); } } @@ -567,8 +641,11 @@ public void updateState(ClusterState state) { private long bestGeneration(Collection operations) { final String repoName = metadata.name(); - return operations.stream().filter(e -> e.repository().equals(repoName)).mapToLong(RepositoryOperation::repositoryStateId) - .max().orElse(RepositoryData.EMPTY_REPO_GEN); + return operations.stream() + .filter(e -> e.repository().equals(repoName)) + .mapToLong(RepositoryOperation::repositoryStateId) + .max() + .orElse(RepositoryData.EMPTY_REPO_GEN); } public ThreadPool threadPool() { @@ -593,13 +670,13 @@ protected BlobContainer blobContainer() { BlobContainer blobContainer = this.blobContainer.get(); if (blobContainer == null) { - synchronized (lock) { - blobContainer = this.blobContainer.get(); - if (blobContainer == null) { - blobContainer = blobStore().blobContainer(basePath()); - this.blobContainer.set(blobContainer); - } - } + synchronized (lock) { + blobContainer = this.blobContainer.get(); + if (blobContainer == null) { + blobContainer = blobStore().blobContainer(basePath()); + this.blobContainer.set(blobContainer); + } + } } return blobContainer; @@ -625,7 +702,7 @@ public BlobStore blobStore() { } catch (RepositoryException e) { throw e; } catch (Exception e) { - throw new RepositoryException(metadata.name(), "cannot create blob store" , e); + throw new RepositoryException(metadata.name(), "cannot create blob store", e); } blobStore.set(store); } @@ -682,8 +759,12 @@ public RepositoryStats stats() { } @Override - public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + public void deleteSnapshots( + Collection snapshotIds, + long repositoryStateId, + Version repositoryMetaVersion, + ActionListener listener + ) { if (isReadOnly()) { listener.onFailure(new RepositoryException(metadata.name(), "cannot delete snapshot from a readonly repository")); } else { @@ -695,8 +776,15 @@ protected void doRun() throws Exception { // Cache the indices that were found before writing out the new index-N blob so that a stuck master will never // delete an index that was created by another master node after writing this index-N blob. final Map foundIndices = blobStore().blobContainer(indicesPath()).children(); - doDeleteShardSnapshots(snapshotIds, repositoryStateId, foundIndices, rootBlobs, repositoryData, - repositoryMetaVersion, listener); + doDeleteShardSnapshots( + snapshotIds, + repositoryStateId, + foundIndices, + rootBlobs, + repositoryData, + repositoryMetaVersion, + listener + ); } @Override @@ -729,12 +817,24 @@ private RepositoryData safeRepositoryData(long repositoryStateId, Map snapshotIds, long repositoryStateId, Map foundIndices, - Map rootBlobs, RepositoryData repositoryData, Version repoMetaVersion, - ActionListener listener) { + private void doDeleteShardSnapshots( + Collection snapshotIds, + long repositoryStateId, + Map foundIndices, + Map rootBlobs, + RepositoryData repositoryData, + Version repoMetaVersion, + ActionListener listener + ) { if (SnapshotsService.useShardGenerations(repoMetaVersion)) { // First write the new shard state metadata (with the removed snapshot) and compute deletion targets @@ -768,8 +874,8 @@ private void doDeleteShardSnapshots(Collection snapshotIds, long rep // 2. Update the index shard generations of all updated shard folders // // Note: If we fail updating any of the individual shard paths, none of them are changed since the newly created - // index-${gen_uuid} will not be referenced by the existing RepositoryData and new RepositoryData is only - // written if all shard paths have been successfully updated. + // index-${gen_uuid} will not be referenced by the existing RepositoryData and new RepositoryData is only + // written if all shard paths have been successfully updated. final StepListener writeUpdatedRepoDataStep = new StepListener<>(); writeShardMetaDataAndComputeDeletesStep.whenComplete(deleteResults -> { final ShardGenerations.Builder builder = ShardGenerations.builder(); @@ -777,67 +883,88 @@ private void doDeleteShardSnapshots(Collection snapshotIds, long rep builder.put(newGen.indexId, newGen.shardId, newGen.newGeneration); } final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, builder.build()); - writeIndexGen(updatedRepoData, repositoryStateId, repoMetaVersion, Function.identity(), - ActionListener.wrap(writeUpdatedRepoDataStep::onResponse, listener::onFailure)); + writeIndexGen( + updatedRepoData, + repositoryStateId, + repoMetaVersion, + Function.identity(), + ActionListener.wrap(writeUpdatedRepoDataStep::onResponse, listener::onFailure) + ); }, listener::onFailure); // Once we have updated the repository, run the clean-ups writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion - final ActionListener afterCleanupsListener = - new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), 2); + final ActionListener afterCleanupsListener = new GroupedActionListener<>( + ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), + 2 + ); cleanupUnlinkedRootAndIndicesBlobs(snapshotIds, foundIndices, rootBlobs, updatedRepoData, afterCleanupsListener); - asyncCleanupUnlinkedShardLevelBlobs(repositoryData, snapshotIds, writeShardMetaDataAndComputeDeletesStep.result(), - afterCleanupsListener); + asyncCleanupUnlinkedShardLevelBlobs( + repositoryData, + snapshotIds, + writeShardMetaDataAndComputeDeletesStep.result(), + afterCleanupsListener + ); }, listener::onFailure); } else { // Write the new repository data first (with the removed snapshot), using no shard generations final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, ShardGenerations.EMPTY); writeIndexGen(updatedRepoData, repositoryStateId, repoMetaVersion, Function.identity(), ActionListener.wrap(newRepoData -> { // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion - final ActionListener afterCleanupsListener = - new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(newRepoData)), 2); + final ActionListener afterCleanupsListener = new GroupedActionListener<>( + ActionListener.wrap(() -> listener.onResponse(newRepoData)), + 2 + ); cleanupUnlinkedRootAndIndicesBlobs(snapshotIds, foundIndices, rootBlobs, newRepoData, afterCleanupsListener); final StepListener> writeMetaAndComputeDeletesStep = new StepListener<>(); writeUpdatedShardMetaDataAndComputeDeletes(snapshotIds, repositoryData, false, writeMetaAndComputeDeletesStep); - writeMetaAndComputeDeletesStep.whenComplete(deleteResults -> - asyncCleanupUnlinkedShardLevelBlobs(repositoryData, snapshotIds, deleteResults, afterCleanupsListener), - afterCleanupsListener::onFailure); + writeMetaAndComputeDeletesStep.whenComplete( + deleteResults -> asyncCleanupUnlinkedShardLevelBlobs(repositoryData, snapshotIds, deleteResults, afterCleanupsListener), + afterCleanupsListener::onFailure + ); }, listener::onFailure)); } } - private void cleanupUnlinkedRootAndIndicesBlobs(Collection deletedSnapshots, Map foundIndices, - Map rootBlobs, RepositoryData updatedRepoData, - ActionListener listener) { + private void cleanupUnlinkedRootAndIndicesBlobs( + Collection deletedSnapshots, + Map foundIndices, + Map rootBlobs, + RepositoryData updatedRepoData, + ActionListener listener + ) { cleanupStaleBlobs(deletedSnapshots, foundIndices, rootBlobs, updatedRepoData, listener.map(ignored -> null)); } - private void asyncCleanupUnlinkedShardLevelBlobs(RepositoryData oldRepositoryData, Collection snapshotIds, - Collection deleteResults, - ActionListener listener) { + private void asyncCleanupUnlinkedShardLevelBlobs( + RepositoryData oldRepositoryData, + Collection snapshotIds, + Collection deleteResults, + ActionListener listener + ) { final Iterator filesToDelete = resolveFilesToDelete(oldRepositoryData, snapshotIds, deleteResults); if (filesToDelete.hasNext() == false) { listener.onResponse(null); return; } - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap( - listener, - l -> { - try { - deleteFromContainer(blobContainer(), filesToDelete); - l.onResponse(null); - } catch (Exception e) { - logger.warn( - () -> new ParameterizedMessage("{} Failed to delete some blobs during snapshot delete", snapshotIds), - e); - throw e; - } - })); + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(listener, l -> { + try { + deleteFromContainer(blobContainer(), filesToDelete); + l.onResponse(null); + } catch (Exception e) { + logger.warn(() -> new ParameterizedMessage("{} Failed to delete some blobs during snapshot delete", snapshotIds), e); + throw e; + } + })); } // updates the shard state metadata for shards of a snapshot that is to be deleted. Also computes the files to be cleaned up. - private void writeUpdatedShardMetaDataAndComputeDeletes(Collection snapshotIds, RepositoryData oldRepositoryData, - boolean useUUIDs, ActionListener> onAllShardsCompleted) { + private void writeUpdatedShardMetaDataAndComputeDeletes( + Collection snapshotIds, + RepositoryData oldRepositoryData, + boolean useUUIDs, + ActionListener> onAllShardsCompleted + ) { final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); final List indices = oldRepositoryData.indicesToUpdateAfterRemovingSnapshot(snapshotIds); @@ -849,29 +976,41 @@ private void writeUpdatedShardMetaDataAndComputeDeletes(Collection s // Listener that flattens out the delete results for each index final ActionListener> deleteIndexMetadataListener = new GroupedActionListener<>( - onAllShardsCompleted.map(res -> res.stream().flatMap(Collection::stream).collect(Collectors.toList())), indices.size()); + onAllShardsCompleted.map(res -> res.stream().flatMap(Collection::stream).collect(Collectors.toList())), + indices.size() + ); for (IndexId indexId : indices) { - final Set survivingSnapshots = oldRepositoryData.getSnapshots(indexId).stream() - .filter(id -> snapshotIds.contains(id) == false).collect(Collectors.toSet()); + final Set survivingSnapshots = oldRepositoryData.getSnapshots(indexId) + .stream() + .filter(id -> snapshotIds.contains(id) == false) + .collect(Collectors.toSet()); final StepListener> shardCountListener = new StepListener<>(); - final Collection indexMetaGenerations = snapshotIds.stream().map( - id -> oldRepositoryData.indexMetaDataGenerations().indexMetaBlobId(id, indexId)).collect(Collectors.toSet()); - final ActionListener allShardCountsListener = - new GroupedActionListener<>(shardCountListener, indexMetaGenerations.size()); + final Collection indexMetaGenerations = snapshotIds.stream() + .map(id -> oldRepositoryData.indexMetaDataGenerations().indexMetaBlobId(id, indexId)) + .collect(Collectors.toSet()); + final ActionListener allShardCountsListener = new GroupedActionListener<>( + shardCountListener, + indexMetaGenerations.size() + ); final BlobContainer indexContainer = indexContainer(indexId); for (String indexMetaGeneration : indexMetaGenerations) { executor.execute(ActionRunnable.supply(allShardCountsListener, () -> { try { - return INDEX_METADATA_FORMAT.read(indexContainer, indexMetaGeneration, namedXContentRegistry - ).getNumberOfShards(); + return INDEX_METADATA_FORMAT.read(indexContainer, indexMetaGeneration, namedXContentRegistry).getNumberOfShards(); } catch (Exception ex) { - logger.warn(() -> new ParameterizedMessage( - "[{}] [{}] failed to read metadata for index", indexMetaGeneration, indexId.getName()), ex); + logger.warn( + () -> new ParameterizedMessage( + "[{}] [{}] failed to read metadata for index", + indexMetaGeneration, + indexId.getName() + ), + ex + ); // Just invoke the listener without any shard generations to count it down, this index will be cleaned up // by the stale data cleanup in the end. // TODO: Getting here means repository corruption. We should find a way of dealing with this instead of just - // ignoring it and letting the cleanup deal with it. + // ignoring it and letting the cleanup deal with it. return null; } })); @@ -883,8 +1022,10 @@ private void writeUpdatedShardMetaDataAndComputeDeletes(Collection s return; } // Listener for collecting the results of removing the snapshot from each shard's metadata in the current index - final ActionListener allShardsListener = - new GroupedActionListener<>(deleteIndexMetadataListener, shardCount); + final ActionListener allShardsListener = new GroupedActionListener<>( + deleteIndexMetadataListener, + shardCount + ); for (int shardId = 0; shardId < shardCount; shardId++) { final int finalShardId = shardId; executor.execute(new AbstractRunnable() { @@ -896,22 +1037,41 @@ protected void doRun() throws Exception { final long newGen; if (useUUIDs) { newGen = -1L; - blobStoreIndexShardSnapshots = buildBlobStoreIndexShardSnapshots(blobs, shardContainer, - oldRepositoryData.shardGenerations().getShardGen(indexId, finalShardId)).v1(); + blobStoreIndexShardSnapshots = buildBlobStoreIndexShardSnapshots( + blobs, + shardContainer, + oldRepositoryData.shardGenerations().getShardGen(indexId, finalShardId) + ).v1(); } else { Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs, shardContainer); newGen = tuple.v2() + 1; blobStoreIndexShardSnapshots = tuple.v1(); } - allShardsListener.onResponse(deleteFromShardSnapshotMeta(survivingSnapshots, indexId, finalShardId, - snapshotIds, shardContainer, blobs, blobStoreIndexShardSnapshots, newGen)); + allShardsListener.onResponse( + deleteFromShardSnapshotMeta( + survivingSnapshots, + indexId, + finalShardId, + snapshotIds, + shardContainer, + blobs, + blobStoreIndexShardSnapshots, + newGen + ) + ); } @Override public void onFailure(Exception ex) { logger.warn( - () -> new ParameterizedMessage("{} failed to delete shard data for shard [{}][{}]", - snapshotIds, indexId.getName(), finalShardId), ex); + () -> new ParameterizedMessage( + "{} failed to delete shard data for shard [{}][{}]", + snapshotIds, + indexId.getName(), + finalShardId + ), + ex + ); // Just passing null here to count down the listener instead of failing it, the stale data left behind // here will be retried in the next delete or repository cleanup allShardsListener.onResponse(null); @@ -922,23 +1082,23 @@ public void onFailure(Exception ex) { } } - private Iterator resolveFilesToDelete(RepositoryData oldRepositoryData, Collection snapshotIds, - Collection deleteResults) { + private Iterator resolveFilesToDelete( + RepositoryData oldRepositoryData, + Collection snapshotIds, + Collection deleteResults + ) { final String basePath = basePath().buildAsString(); final int basePathLen = basePath.length(); - final Map> indexMetaGenerations = - oldRepositoryData.indexMetaDataToRemoveAfterRemovingSnapshots(snapshotIds); - return Stream.concat( - deleteResults.stream().flatMap(shardResult -> { - final String shardPath = - shardContainer(shardResult.indexId, shardResult.shardId).path().buildAsString(); - return shardResult.blobsToDelete.stream().map(blob -> shardPath + blob); - }), - indexMetaGenerations.entrySet().stream().flatMap(entry -> { - final String indexContainerPath = indexContainer(entry.getKey()).path().buildAsString(); - return entry.getValue().stream().map(id -> indexContainerPath + INDEX_METADATA_FORMAT.blobName(id)); - }) - ).map(absolutePath -> { + final Map> indexMetaGenerations = oldRepositoryData.indexMetaDataToRemoveAfterRemovingSnapshots( + snapshotIds + ); + return Stream.concat(deleteResults.stream().flatMap(shardResult -> { + final String shardPath = shardContainer(shardResult.indexId, shardResult.shardId).path().buildAsString(); + return shardResult.blobsToDelete.stream().map(blob -> shardPath + blob); + }), indexMetaGenerations.entrySet().stream().flatMap(entry -> { + final String indexContainerPath = indexContainer(entry.getKey()).path().buildAsString(); + return entry.getValue().stream().map(id -> indexContainerPath + INDEX_METADATA_FORMAT.blobName(id)); + })).map(absolutePath -> { assert absolutePath.startsWith(basePath); return absolutePath.substring(basePathLen); }).iterator(); @@ -956,9 +1116,13 @@ private Iterator resolveFilesToDelete(RepositoryData oldRepositoryData, * @param newRepoData new repository data that was just written * @param listener listener to invoke with the combined {@link DeleteResult} of all blobs removed in this operation */ - private void cleanupStaleBlobs(Collection deletedSnapshots, Map foundIndices, - Map rootBlobs, RepositoryData newRepoData, - ActionListener listener) { + private void cleanupStaleBlobs( + Collection deletedSnapshots, + Map foundIndices, + Map rootBlobs, + RepositoryData newRepoData, + ActionListener listener + ) { final GroupedActionListener groupedListener = new GroupedActionListener<>(ActionListener.wrap(deleteResults -> { DeleteResult deleteResult = DeleteResult.ZERO; for (DeleteResult result : deleteResults) { @@ -973,8 +1137,7 @@ private void cleanupStaleBlobs(Collection deletedSnapshots, Map { - List deletedBlobs = - cleanupStaleRootFiles(newRepoData.getGenId() - 1, deletedSnapshots, staleRootBlobs); + List deletedBlobs = cleanupStaleRootFiles(newRepoData.getGenId() - 1, deletedSnapshots, staleRootBlobs); return new DeleteResult(deletedBlobs.size(), deletedBlobs.stream().mapToLong(name -> rootBlobs.get(name).length()).sum()); })); } @@ -1008,17 +1171,33 @@ public void cleanup(long repositoryStateId, Version repositoryMetaVersion, Actio Map rootBlobs = blobContainer().listBlobs(); final RepositoryData repositoryData = safeRepositoryData(repositoryStateId, rootBlobs); final Map foundIndices = blobStore().blobContainer(indicesPath()).children(); - final Set survivingIndexIds = - repositoryData.getIndices().values().stream().map(IndexId::getId).collect(Collectors.toSet()); + final Set survivingIndexIds = repositoryData.getIndices() + .values() + .stream() + .map(IndexId::getId) + .collect(Collectors.toSet()); final List staleRootBlobs = staleRootBlobs(repositoryData, rootBlobs.keySet()); if (survivingIndexIds.equals(foundIndices.keySet()) && staleRootBlobs.isEmpty()) { // Nothing to clean up we return listener.onResponse(new RepositoryCleanupResult(DeleteResult.ZERO)); } else { // write new index-N blob to ensure concurrent operations will fail - writeIndexGen(repositoryData, repositoryStateId, repositoryMetaVersion, - Function.identity(), ActionListener.wrap(v -> cleanupStaleBlobs(Collections.emptyList(), foundIndices, rootBlobs, - repositoryData, listener.map(RepositoryCleanupResult::new)), listener::onFailure)); + writeIndexGen( + repositoryData, + repositoryStateId, + repositoryMetaVersion, + Function.identity(), + ActionListener.wrap( + v -> cleanupStaleBlobs( + Collections.emptyList(), + foundIndices, + rootBlobs, + repositoryData, + listener.map(RepositoryCleanupResult::new) + ), + listener::onFailure + ) + ); } } catch (Exception e) { listener.onFailure(e); @@ -1027,36 +1206,36 @@ public void cleanup(long repositoryStateId, Version repositoryMetaVersion, Actio // Finds all blobs directly under the repository root path that are not referenced by the current RepositoryData private static List staleRootBlobs(RepositoryData repositoryData, Set rootBlobNames) { - final Set allSnapshotIds = - repositoryData.getSnapshotIds().stream().map(SnapshotId::getUUID).collect(Collectors.toSet()); - return rootBlobNames.stream().filter( - blob -> { - if (FsBlobContainer.isTempBlobName(blob)) { - return true; - } - if (blob.endsWith(".dat")) { - final String foundUUID; - if (blob.startsWith(SNAPSHOT_PREFIX)) { - foundUUID = blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length()); - assert SNAPSHOT_FORMAT.blobName(foundUUID).equals(blob); - } else if (blob.startsWith(METADATA_PREFIX)) { - foundUUID = blob.substring(METADATA_PREFIX.length(), blob.length() - ".dat".length()); - assert GLOBAL_METADATA_FORMAT.blobName(foundUUID).equals(blob); - } else { - return false; - } - return allSnapshotIds.contains(foundUUID) == false; - } else if (blob.startsWith(INDEX_FILE_PREFIX)) { - // TODO: Include the current generation here once we remove keeping index-(N-1) around from #writeIndexGen - return repositoryData.getGenId() > Long.parseLong(blob.substring(INDEX_FILE_PREFIX.length())); + final Set allSnapshotIds = repositoryData.getSnapshotIds().stream().map(SnapshotId::getUUID).collect(Collectors.toSet()); + return rootBlobNames.stream().filter(blob -> { + if (FsBlobContainer.isTempBlobName(blob)) { + return true; + } + if (blob.endsWith(".dat")) { + final String foundUUID; + if (blob.startsWith(SNAPSHOT_PREFIX)) { + foundUUID = blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length()); + assert SNAPSHOT_FORMAT.blobName(foundUUID).equals(blob); + } else if (blob.startsWith(METADATA_PREFIX)) { + foundUUID = blob.substring(METADATA_PREFIX.length(), blob.length() - ".dat".length()); + assert GLOBAL_METADATA_FORMAT.blobName(foundUUID).equals(blob); + } else { + return false; } - return false; + return allSnapshotIds.contains(foundUUID) == false; + } else if (blob.startsWith(INDEX_FILE_PREFIX)) { + // TODO: Include the current generation here once we remove keeping index-(N-1) around from #writeIndexGen + return repositoryData.getGenId() > Long.parseLong(blob.substring(INDEX_FILE_PREFIX.length())); } - ).collect(Collectors.toList()); + return false; + }).collect(Collectors.toList()); } - private List cleanupStaleRootFiles(long previousGeneration, Collection deletedSnapshots, - List blobsToDelete) { + private List cleanupStaleRootFiles( + long previousGeneration, + Collection deletedSnapshots, + List blobsToDelete + ) { if (blobsToDelete.isEmpty()) { return blobsToDelete; } @@ -1065,12 +1244,18 @@ private List cleanupStaleRootFiles(long previousGeneration, Collection blobNamesToIgnore = deletedSnapshots.stream().flatMap( - snapshotId -> Stream.of(GLOBAL_METADATA_FORMAT.blobName(snapshotId.getUUID()), - SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()), INDEX_FILE_PREFIX + previousGeneration)) - .collect(Collectors.toSet()); - final List blobsToLog = blobsToDelete.stream().filter(b -> blobNamesToIgnore.contains(b) == false) - .collect(Collectors.toList()); + final Set blobNamesToIgnore = deletedSnapshots.stream() + .flatMap( + snapshotId -> Stream.of( + GLOBAL_METADATA_FORMAT.blobName(snapshotId.getUUID()), + SNAPSHOT_FORMAT.blobName(snapshotId.getUUID()), + INDEX_FILE_PREFIX + previousGeneration + ) + ) + .collect(Collectors.toSet()); + final List blobsToLog = blobsToDelete.stream() + .filter(b -> blobNamesToIgnore.contains(b) == false) + .collect(Collectors.toList()); if (blobsToLog.isEmpty() == false) { logger.info("[{}] Found stale root level blobs {}. Cleaning them up", metadata.name(), blobsToLog); } @@ -1078,9 +1263,14 @@ private List cleanupStaleRootFiles(long previousGeneration, Collection new ParameterizedMessage( - "[{}] The following blobs are no longer part of any snapshot [{}] but failed to remove them", - metadata.name(), blobsToDelete), e); + logger.warn( + () -> new ParameterizedMessage( + "[{}] The following blobs are no longer part of any snapshot [{}] but failed to remove them", + metadata.name(), + blobsToDelete + ), + e + ); } return Collections.emptyList(); } @@ -1096,24 +1286,31 @@ private DeleteResult cleanupStaleIndices(Map foundIndices logger.debug("[{}] Cleaned up stale index [{}]", metadata.name(), indexSnId); } } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage( - "[{}] index {} is no longer part of any snapshot in the repository, " + - "but failed to clean up its index folder", metadata.name(), indexSnId), e); + logger.warn( + () -> new ParameterizedMessage( + "[{}] index {} is no longer part of any snapshot in the repository, " + "but failed to clean up its index folder", + metadata.name(), + indexSnId + ), + e + ); } } return deleteResult; } @Override - public void finalizeSnapshot(final ShardGenerations shardGenerations, - final long repositoryStateId, - final Metadata clusterMetadata, - SnapshotInfo snapshotInfo, - Version repositoryMetaVersion, - Function stateTransformer, - final ActionListener listener) { - assert repositoryStateId > RepositoryData.UNKNOWN_REPO_GEN : - "Must finalize based on a valid repository generation but received [" + repositoryStateId + "]"; + public void finalizeSnapshot( + final ShardGenerations shardGenerations, + final long repositoryStateId, + final Metadata clusterMetadata, + SnapshotInfo snapshotInfo, + Version repositoryMetaVersion, + Function stateTransformer, + final ActionListener listener + ) { + assert repositoryStateId > RepositoryData.UNKNOWN_REPO_GEN + : "Must finalize based on a valid repository generation but received [" + repositoryStateId + "]"; final Collection indices = shardGenerations.indices(); final SnapshotId snapshotId = snapshotInfo.snapshotId(); // Once we are done writing the updated index-N blob we remove the now unreferenced index-${uuid} blobs in each shard @@ -1121,8 +1318,9 @@ public void finalizeSnapshot(final ShardGenerations shardGenerations, // If there are older version nodes in the cluster, we don't need to run this cleanup as it will have already happened // when writing the index-${N} to each shard directory. final boolean writeShardGens = SnapshotsService.useShardGenerations(repositoryMetaVersion); - final Consumer onUpdateFailure = - e -> listener.onFailure(new SnapshotException(metadata.name(), snapshotId, "failed to update snapshot in repository", e)); + final Consumer onUpdateFailure = e -> listener.onFailure( + new SnapshotException(metadata.name(), snapshotId, "failed to update snapshot in repository", e) + ); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); @@ -1133,9 +1331,17 @@ public void finalizeSnapshot(final ShardGenerations shardGenerations, repoDataListener.whenComplete(existingRepositoryData -> { final int existingSnapshotCount = existingRepositoryData.getSnapshotIds().size(); if (existingSnapshotCount >= maxSnapshotCount) { - listener.onFailure(new RepositoryException(metadata.name(), "Cannot add another snapshot to this repository as it " + - "already contains [" + existingSnapshotCount + "] snapshots and is configured to hold up to [" + maxSnapshotCount + - "] snapshots only.")); + listener.onFailure( + new RepositoryException( + metadata.name(), + "Cannot add another snapshot to this repository as it " + + "already contains [" + + existingSnapshotCount + + "] snapshots and is configured to hold up to [" + + maxSnapshotCount + + "] snapshots only." + ) + ); return; } @@ -1149,24 +1355,33 @@ public void finalizeSnapshot(final ShardGenerations shardGenerations, indexMetaIdentifiers = null; } - final ActionListener allMetaListener = new GroupedActionListener<>( - ActionListener.wrap(v -> { - final SnapshotDetails snapshotDetails = new SnapshotDetails( - snapshotInfo.state(), - Version.CURRENT, - snapshotInfo.startTime(), - snapshotInfo.endTime()); - final RepositoryData updatedRepositoryData = existingRepositoryData.addSnapshot( - snapshotId, snapshotDetails, shardGenerations, indexMetas, indexMetaIdentifiers); - writeIndexGen(updatedRepositoryData, repositoryStateId, repositoryMetaVersion, stateTransformer, - ActionListener.wrap( - newRepoData -> { - if (writeShardGens) { - cleanupOldShardGens(existingRepositoryData, updatedRepositoryData); - } - listener.onResponse(newRepoData); - }, onUpdateFailure)); - }, onUpdateFailure), 2 + indices.size()); + final ActionListener allMetaListener = new GroupedActionListener<>(ActionListener.wrap(v -> { + final SnapshotDetails snapshotDetails = new SnapshotDetails( + snapshotInfo.state(), + Version.CURRENT, + snapshotInfo.startTime(), + snapshotInfo.endTime() + ); + final RepositoryData updatedRepositoryData = existingRepositoryData.addSnapshot( + snapshotId, + snapshotDetails, + shardGenerations, + indexMetas, + indexMetaIdentifiers + ); + writeIndexGen( + updatedRepositoryData, + repositoryStateId, + repositoryMetaVersion, + stateTransformer, + ActionListener.wrap(newRepoData -> { + if (writeShardGens) { + cleanupOldShardGens(existingRepositoryData, updatedRepositoryData); + } + listener.onResponse(newRepoData); + }, onUpdateFailure) + ); + }, onUpdateFailure), 2 + indices.size()); // We ignore all FileAlreadyExistsException when writing metadata since otherwise a master failover while in this method will // mean that no snap-${uuid}.dat blob is ever written for this snapshot. This is safe because any updated version of the @@ -1175,32 +1390,44 @@ public void finalizeSnapshot(final ShardGenerations shardGenerations, // that decrements the generation it points at // Write Global MetaData - executor.execute(ActionRunnable.run(allMetaListener, - () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress, bigArrays))); + executor.execute( + ActionRunnable.run( + allMetaListener, + () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress, bigArrays) + ) + ); // write the index metadata for each index in the snapshot for (IndexId index : indices) { executor.execute(ActionRunnable.run(allMetaListener, () -> { - final IndexMetadata indexMetaData = clusterMetadata.index(index.getName()); - if (writeIndexGens) { - final String identifiers = IndexMetaDataGenerations.buildUniqueIdentifier(indexMetaData); - String metaUUID = existingRepositoryData.indexMetaDataGenerations().getIndexMetaBlobId(identifiers); - if (metaUUID == null) { - // We don't yet have this version of the metadata so we write it - metaUUID = UUIDs.base64UUID(); - INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress, bigArrays); - indexMetaIdentifiers.put(identifiers, metaUUID); - } - indexMetas.put(index, identifiers); - } else { - INDEX_METADATA_FORMAT.write( - clusterMetadata.index(index.getName()), indexContainer(index), snapshotId.getUUID(), compress, bigArrays); + final IndexMetadata indexMetaData = clusterMetadata.index(index.getName()); + if (writeIndexGens) { + final String identifiers = IndexMetaDataGenerations.buildUniqueIdentifier(indexMetaData); + String metaUUID = existingRepositoryData.indexMetaDataGenerations().getIndexMetaBlobId(identifiers); + if (metaUUID == null) { + // We don't yet have this version of the metadata so we write it + metaUUID = UUIDs.base64UUID(); + INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress, bigArrays); + indexMetaIdentifiers.put(identifiers, metaUUID); } + indexMetas.put(index, identifiers); + } else { + INDEX_METADATA_FORMAT.write( + clusterMetadata.index(index.getName()), + indexContainer(index), + snapshotId.getUUID(), + compress, + bigArrays + ); } - )); + })); } - executor.execute(ActionRunnable.run(allMetaListener, - () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress, bigArrays))); + executor.execute( + ActionRunnable.run( + allMetaListener, + () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress, bigArrays) + ) + ); }, onUpdateFailure); } @@ -1208,9 +1435,15 @@ public void finalizeSnapshot(final ShardGenerations shardGenerations, private void cleanupOldShardGens(RepositoryData existingRepositoryData, RepositoryData updatedRepositoryData) { final List toDelete = new ArrayList<>(); final int prefixPathLen = basePath().buildAsString().length(); - updatedRepositoryData.shardGenerations().obsoleteShardGenerations(existingRepositoryData.shardGenerations()).forEach( - (indexId, gens) -> gens.forEach((shardId, oldGen) -> toDelete.add( - shardContainer(indexId, shardId).path().buildAsString().substring(prefixPathLen) + INDEX_FILE_PREFIX + oldGen))); + updatedRepositoryData.shardGenerations() + .obsoleteShardGenerations(existingRepositoryData.shardGenerations()) + .forEach( + (indexId, gens) -> gens.forEach( + (shardId, oldGen) -> toDelete.add( + shardContainer(indexId, shardId).path().buildAsString().substring(prefixPathLen) + INDEX_FILE_PREFIX + oldGen + ) + ) + ); try { deleteFromContainer(blobContainer(), toDelete.iterator()); } catch (Exception e) { @@ -1256,7 +1489,8 @@ private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfo failure = new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex); } catch (Exception e) { failure = e instanceof SnapshotException - ? e : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e); + ? e + : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e); } if (failure != null) { if (context.failFast()) { @@ -1285,8 +1519,11 @@ public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) { @Override public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, IndexId index) throws IOException { try { - return INDEX_METADATA_FORMAT.read(indexContainer(index), - repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index), namedXContentRegistry); + return INDEX_METADATA_FORMAT.read( + indexContainer(index), + repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index), + namedXContentRegistry + ); } catch (NoSuchFileException e) { throw new SnapshotMissingException(metadata.name(), snapshotId, e); } @@ -1359,8 +1596,8 @@ public long getRestoreThrottleTimeInNanos() { protected void assertSnapshotOrGenericThread() { assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT + ']') || Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT_META + ']') - || Thread.currentThread().getName().contains('[' + ThreadPool.Names.GENERIC + ']') : - "Expected current thread [" + Thread.currentThread() + "] to be the snapshot or generic thread."; + || Thread.currentThread().getName().contains('[' + ThreadPool.Names.GENERIC + ']') + : "Expected current thread [" + Thread.currentThread() + "] to be the snapshot or generic thread."; } @Override @@ -1421,12 +1658,18 @@ public void getRepositoryData(ActionListener listener) { return; } if (metadata.generation() == RepositoryData.UNKNOWN_REPO_GEN && isReadOnly() == false) { - logger.debug("[{}] loading repository metadata for the first time, trying to determine correct generation and to store " + - "it in the cluster state", metadata.name()); + logger.debug( + "[{}] loading repository metadata for the first time, trying to determine correct generation and to store " + + "it in the cluster state", + metadata.name() + ); initializeRepoGenerationTracking(listener); } else { - logger.trace("[{}] loading un-cached repository data with best known repository generation [{}]", metadata.name(), - latestKnownRepoGen); + logger.trace( + "[{}] loading un-cached repository data with best known repository generation [{}]", + metadata.name(), + latestKnownRepoGen + ); // Don't deduplicate repo data loading if we don't have strong consistency guarantees between the repo and the cluster state // Also, if we are not caching repository data (for tests) we assume that the contents of the repository data at a given // generation may change @@ -1434,8 +1677,11 @@ public void getRepositoryData(ActionListener listener) { if (bestEffortConsistency || cacheRepositoryData == false) { executor.execute(ActionRunnable.wrap(listener, this::doGetRepositoryData)); } else { - repoDataDeduplicator.executeOnce(metadata, listener, (metadata, l) -> - executor.execute(ActionRunnable.wrap(l, this::doGetRepositoryData))); + repoDataDeduplicator.executeOnce( + metadata, + listener, + (metadata, l) -> executor.execute(ActionRunnable.wrap(l, this::doGetRepositoryData)) + ); } } } @@ -1465,8 +1711,13 @@ private void initializeRepoGenerationTracking(ActionListener lis repoDataInitialized = new ListenableActionFuture<>(); repoDataInitialized.addListener(listener); final Consumer onFailure = e -> { - logger.warn(new ParameterizedMessage("[{}] Exception when initializing repository generation in cluster state", - metadata.name()), e); + logger.warn( + new ParameterizedMessage( + "[{}] Exception when initializing repository generation in cluster state", + metadata.name() + ), + e + ); final ActionListener existingListener; synchronized (BlobStoreRepository.this) { existingListener = repoDataInitialized; @@ -1474,53 +1725,83 @@ private void initializeRepoGenerationTracking(ActionListener lis } existingListener.onFailure(e); }; - threadPool.generic().execute(ActionRunnable.wrap( - ActionListener.wrap(repoData -> clusterService.submitStateUpdateTask( - "set initial safe repository generation [" + metadata.name() + "][" + repoData.getGenId() + "]", - new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - RepositoryMetadata metadata = getRepoMetadata(currentState); - // No update to the repository generation should have occurred concurrently in general except for - // extreme corner cases like failing over to an older version master node and back to the current - // node concurrently - if (metadata.generation() != RepositoryData.UNKNOWN_REPO_GEN) { - throw new RepositoryException( - metadata.name(), "Found unexpected initialized repo metadata [" + metadata + "]"); + threadPool.generic() + .execute( + ActionRunnable.wrap( + ActionListener.wrap( + repoData -> clusterService.submitStateUpdateTask( + "set initial safe repository generation [" + metadata.name() + "][" + repoData.getGenId() + "]", + new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + RepositoryMetadata metadata = getRepoMetadata(currentState); + // No update to the repository generation should have occurred concurrently in general except + // for + // extreme corner cases like failing over to an older version master node and back to the + // current + // node concurrently + if (metadata.generation() != RepositoryData.UNKNOWN_REPO_GEN) { + throw new RepositoryException( + metadata.name(), + "Found unexpected initialized repo metadata [" + metadata + "]" + ); + } + return ClusterState.builder(currentState) + .metadata( + Metadata.builder(currentState.getMetadata()) + .putCustom( + RepositoriesMetadata.TYPE, + currentState.metadata() + .custom(RepositoriesMetadata.TYPE) + .withUpdatedGeneration( + metadata.name(), + repoData.getGenId(), + repoData.getGenId() + ) + ) + ) + .build(); } - return ClusterState.builder(currentState) - .metadata(Metadata.builder(currentState.getMetadata()).putCustom( - RepositoriesMetadata.TYPE, - currentState.metadata().custom(RepositoriesMetadata.TYPE) - .withUpdatedGeneration(metadata.name(), - repoData.getGenId(), repoData.getGenId()))).build(); - } - @Override - public void onFailure(String source, Exception e) { - onFailure.accept(e); - } + @Override + public void onFailure(String source, Exception e) { + onFailure.accept(e); + } - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - logger.trace("[{}] initialized repository generation in cluster state to [{}]", - metadata.name(), repoData.getGenId()); - // Resolve listeners on generic pool since some callbacks for repository data do additional IO - threadPool.generic().execute(() -> { - final ActionListener existingListener; - synchronized (BlobStoreRepository.this) { - existingListener = repoDataInitialized; - repoDataInitialized = null; - } - existingListener.onResponse(repoData); - logger.trace("[{}] called listeners after initializing repository to generation [{}]" - , metadata.name(), repoData.getGenId()); - }); + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + logger.trace( + "[{}] initialized repository generation in cluster state to [{}]", + metadata.name(), + repoData.getGenId() + ); + // Resolve listeners on generic pool since some callbacks for repository data do additional IO + threadPool.generic().execute(() -> { + final ActionListener existingListener; + synchronized (BlobStoreRepository.this) { + existingListener = repoDataInitialized; + repoDataInitialized = null; + } + existingListener.onResponse(repoData); + logger.trace( + "[{}] called listeners after initializing repository to generation [{}]", + metadata.name(), + repoData.getGenId() + ); + }); + } } - }), onFailure), this::doGetRepositoryData)); + ), + onFailure + ), + this::doGetRepositoryData + ) + ); } else { - logger.trace("[{}] waiting for existing initialization of repository metadata generation in cluster state", - metadata.name()); + logger.trace( + "[{}] waiting for existing initialization of repository metadata generation in cluster state", + metadata.name() + ); repoDataInitialized.addListener(listener); } } @@ -1549,13 +1830,17 @@ private void doGetRepositoryData(ActionListener listener) { generation = latestIndexBlobId(); } catch (Exception e) { listener.onFailure( - new RepositoryException(metadata.name(), "Could not determine repository generation from root blobs", e)); + new RepositoryException(metadata.name(), "Could not determine repository generation from root blobs", e) + ); return; } genToLoad = latestKnownRepoGen.updateAndGet(known -> Math.max(known, generation)); if (genToLoad > generation) { - logger.info("Determined repository generation [{}] from repository contents but correct generation must be at " + - "least [{}]", generation, genToLoad); + logger.info( + "Determined repository generation [{}] from repository contents but correct generation must be at " + "least [{}]", + generation, + genToLoad + ); } } else { // We only rely on the generation tracked in #latestKnownRepoGen which is exclusively updated from the cluster state @@ -1579,15 +1864,11 @@ private void doGetRepositoryData(ActionListener listener) { } else { // someone switched the repo contents out from under us RepositoriesService.updateRepositoryUuidInMetadata( - clusterService, - metadata.name(), - loaded, - new ThreadedActionListener<>( - logger, - threadPool, - ThreadPool.Names.GENERIC, - listener.map(v -> loaded), - false)); + clusterService, + metadata.name(), + loaded, + new ThreadedActionListener<>(logger, threadPool, ThreadPool.Names.GENERIC, listener.map(v -> loaded), false) + ); } } return; @@ -1595,8 +1876,14 @@ private void doGetRepositoryData(ActionListener listener) { // If the generation to load changed concurrently and we didn't just try loading the same generation before we retry if (genToLoad != latestKnownRepoGen.get() && genToLoad != lastFailedGeneration) { lastFailedGeneration = genToLoad; - logger.warn("Failed to load repository data generation [" + genToLoad + - "] because a concurrent operation moved the current generation to [" + latestKnownRepoGen.get() + "]", e); + logger.warn( + "Failed to load repository data generation [" + + genToLoad + + "] because a concurrent operation moved the current generation to [" + + latestKnownRepoGen.get() + + "]", + e + ); continue; } if (bestEffortConsistency == false && ExceptionsHelper.unwrap(e, NoSuchFileException.class) != null) { @@ -1609,8 +1896,11 @@ private void doGetRepositoryData(ActionListener listener) { e.addSuppressed(ex); } final Tuple finalLastInfo = previousWriterInformation; - markRepoCorrupted(genToLoad, e, - ActionListener.wrap(v -> listener.onFailure(corruptedStateException(e, finalLastInfo)), listener::onFailure)); + markRepoCorrupted( + genToLoad, + e, + ActionListener.wrap(v -> listener.onFailure(corruptedStateException(e, finalLastInfo)), listener::onFailure) + ); } else { listener.onFailure(e); } @@ -1638,9 +1928,12 @@ private void cacheRepositoryData(RepositoryData repositoryData, Version version) } else { // don't cache shard generations here as they may be unreliable toCache = repositoryData.withoutShardGenerations(); - assert repositoryData.indexMetaDataGenerations().equals(IndexMetaDataGenerations.EMPTY) : - "repository data should not contain index generations at version [" + version + "] but saw [" - + repositoryData.indexMetaDataGenerations() + "]"; + assert repositoryData.indexMetaDataGenerations().equals(IndexMetaDataGenerations.EMPTY) + : "repository data should not contain index generations at version [" + + version + + "] but saw [" + + repositoryData.indexMetaDataGenerations() + + "]"; } assert toCache.getGenId() >= 0 : "No need to cache abstract generations but attempted to cache [" + toCache.getGenId() + "]"; latestKnownRepositoryData.updateAndGet(known -> { @@ -1652,20 +1945,27 @@ private void cacheRepositoryData(RepositoryData repositoryData, Version version) } private RepositoryException corruptedStateException(@Nullable Exception cause, @Nullable Tuple previousWriterInfo) { - return new RepositoryException(metadata.name(), - "Could not read repository data because the contents of the repository do not match its " + - "expected state. This is likely the result of either concurrently modifying the contents of the " + - "repository by a process other than this cluster or an issue with the repository's underlying storage. " + - "The repository has been disabled to prevent corrupting its contents. To re-enable it " + - "and continue using it please remove the repository from the cluster and add it again to make " + - "the cluster recover the known state of the repository from its physical contents." + - previousWriterMessage(previousWriterInfo), - cause); + return new RepositoryException( + metadata.name(), + "Could not read repository data because the contents of the repository do not match its " + + "expected state. This is likely the result of either concurrently modifying the contents of the " + + "repository by a process other than this cluster or an issue with the repository's underlying storage. " + + "The repository has been disabled to prevent corrupting its contents. To re-enable it " + + "and continue using it please remove the repository from the cluster and add it again to make " + + "the cluster recover the known state of the repository from its physical contents." + + previousWriterMessage(previousWriterInfo), + cause + ); } private String previousWriterMessage(@Nullable Tuple previousWriterInfo) { - return previousWriterInfo == null ? "" : " The last cluster to write to this repository was [" + previousWriterInfo.v2() - + "] at generation [" + previousWriterInfo.v1() + "]."; + return previousWriterInfo == null + ? "" + : " The last cluster to write to this repository was [" + + previousWriterInfo.v2() + + "] at generation [" + + previousWriterInfo.v1() + + "]."; } /** @@ -1681,32 +1981,55 @@ private String previousWriterMessage(@Nullable Tuple previousWrite private void markRepoCorrupted(long corruptedGeneration, Exception originalException, ActionListener listener) { assert corruptedGeneration != RepositoryData.UNKNOWN_REPO_GEN; assert bestEffortConsistency == false; - clusterService.submitStateUpdateTask("mark repository corrupted [" + metadata.name() + "][" + corruptedGeneration + "]", + clusterService.submitStateUpdateTask( + "mark repository corrupted [" + metadata.name() + "][" + corruptedGeneration + "]", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { final RepositoriesMetadata state = currentState.metadata().custom(RepositoriesMetadata.TYPE); final RepositoryMetadata repoState = state.repository(metadata.name()); if (repoState.generation() != corruptedGeneration) { - throw new IllegalStateException("Tried to mark repo generation [" + corruptedGeneration - + "] as corrupted but its state concurrently changed to [" + repoState + "]"); + throw new IllegalStateException( + "Tried to mark repo generation [" + + corruptedGeneration + + "] as corrupted but its state concurrently changed to [" + + repoState + + "]" + ); } - return ClusterState.builder(currentState).metadata(Metadata.builder(currentState.metadata()).putCustom( - RepositoriesMetadata.TYPE, state.withUpdatedGeneration( - metadata.name(), RepositoryData.CORRUPTED_REPO_GEN, repoState.pendingGeneration())).build()).build(); + return ClusterState.builder(currentState) + .metadata( + Metadata.builder(currentState.metadata()) + .putCustom( + RepositoriesMetadata.TYPE, + state.withUpdatedGeneration( + metadata.name(), + RepositoryData.CORRUPTED_REPO_GEN, + repoState.pendingGeneration() + ) + ) + .build() + ) + .build(); } @Override public void onFailure(String source, Exception e) { - listener.onFailure(new RepositoryException(metadata.name(), "Failed marking repository state as corrupted", - ExceptionsHelper.useOrSuppress(e, originalException))); + listener.onFailure( + new RepositoryException( + metadata.name(), + "Failed marking repository state as corrupted", + ExceptionsHelper.useOrSuppress(e, originalException) + ) + ); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { listener.onResponse(null); } - }); + } + ); } private RepositoryData getRepositoryData(long indexGen) { @@ -1717,9 +2040,11 @@ private RepositoryData getRepositoryData(long indexGen) { final String snapshotsIndexBlobName = INDEX_FILE_PREFIX + Long.toString(indexGen); // EMPTY is safe here because RepositoryData#fromXContent calls namedObject - try (InputStream blob = blobContainer().readBlob(snapshotsIndexBlobName); - XContentParser parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, blob)) { + try ( + InputStream blob = blobContainer().readBlob(snapshotsIndexBlobName); + XContentParser parser = XContentType.JSON.xContent() + .createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, blob) + ) { return RepositoryData.snapshotsFromXContent(parser, indexGen, true); } } catch (IOException ioe) { @@ -1758,23 +2083,36 @@ public boolean isReadOnly() { * @param stateFilter filter for the last cluster state update executed by this method * @param listener completion listener */ - protected void writeIndexGen(RepositoryData repositoryData, long expectedGen, Version version, - Function stateFilter, ActionListener listener) { + protected void writeIndexGen( + RepositoryData repositoryData, + long expectedGen, + Version version, + Function stateFilter, + ActionListener listener + ) { logger.trace("[{}] writing repository data on top of expected generation [{}]", metadata.name(), expectedGen); assert isReadOnly() == false; // can not write to a read only repository final long currentGen = repositoryData.getGenId(); if (currentGen != expectedGen) { // the index file was updated by a concurrent operation, so we were operating on stale // repository data - listener.onFailure(new RepositoryException(metadata.name(), - "concurrent modification of the index-N file, expected current generation [" + expectedGen + - "], actual current generation [" + currentGen + "]")); + listener.onFailure( + new RepositoryException( + metadata.name(), + "concurrent modification of the index-N file, expected current generation [" + + expectedGen + + "], actual current generation [" + + currentGen + + "]" + ) + ); return; } // Step 1: Set repository generation state to the next possible pending generation final StepListener setPendingStep = new StepListener<>(); - clusterService.submitStateUpdateTask("set pending repository generation [" + metadata.name() + "][" + expectedGen + "]", + clusterService.submitStateUpdateTask( + "set pending repository generation [" + metadata.name() + "][" + expectedGen + "]", new ClusterStateUpdateTask() { private long newGen; @@ -1786,15 +2124,20 @@ public ClusterState execute(ClusterState currentState) { final long genInState = meta.generation(); final boolean uninitializedMeta = meta.generation() == RepositoryData.UNKNOWN_REPO_GEN || bestEffortConsistency; if (uninitializedMeta == false && meta.pendingGeneration() != genInState) { - logger.info("Trying to write new repository data over unfinished write, repo [{}] is at " + - "safe generation [{}] and pending generation [{}]", meta.name(), genInState, meta.pendingGeneration()); + logger.info( + "Trying to write new repository data over unfinished write, repo [{}] is at " + + "safe generation [{}] and pending generation [{}]", + meta.name(), + genInState, + meta.pendingGeneration() + ); } - assert expectedGen == RepositoryData.EMPTY_REPO_GEN || uninitializedMeta - || expectedGen == meta.generation() : - "Expected non-empty generation [" + expectedGen + "] does not match generation tracked in [" + meta + "]"; + assert expectedGen == RepositoryData.EMPTY_REPO_GEN || uninitializedMeta || expectedGen == meta.generation() + : "Expected non-empty generation [" + expectedGen + "] does not match generation tracked in [" + meta + "]"; // If we run into the empty repo generation for the expected gen, the repo is assumed to have been cleared of // all contents by an external process so we reset the safe generation to the empty generation. - final long safeGeneration = expectedGen == RepositoryData.EMPTY_REPO_GEN ? RepositoryData.EMPTY_REPO_GEN + final long safeGeneration = expectedGen == RepositoryData.EMPTY_REPO_GEN + ? RepositoryData.EMPTY_REPO_GEN : (uninitializedMeta ? expectedGen : genInState); // Regardless of whether or not the safe generation has been reset, the pending generation always increments so that // even if a repository has been manually cleared of all contents we will never reuse the same repository generation. @@ -1802,18 +2145,31 @@ public ClusterState execute(ClusterState currentState) { // not offer any consistency guarantees when it comes to overwriting the same blob name with different content. final long nextPendingGen = metadata.pendingGeneration() + 1; newGen = uninitializedMeta ? Math.max(expectedGen + 1, nextPendingGen) : nextPendingGen; - assert newGen > latestKnownRepoGen.get() : "Attempted new generation [" + newGen + - "] must be larger than latest known generation [" + latestKnownRepoGen.get() + "]"; - return ClusterState.builder(currentState).metadata(Metadata.builder(currentState.getMetadata()) - .putCustom(RepositoriesMetadata.TYPE, - currentState.metadata().custom(RepositoriesMetadata.TYPE).withUpdatedGeneration( - repoName, safeGeneration, newGen)).build()).build(); + assert newGen > latestKnownRepoGen.get() + : "Attempted new generation [" + + newGen + + "] must be larger than latest known generation [" + + latestKnownRepoGen.get() + + "]"; + return ClusterState.builder(currentState) + .metadata( + Metadata.builder(currentState.getMetadata()) + .putCustom( + RepositoriesMetadata.TYPE, + currentState.metadata() + .custom(RepositoriesMetadata.TYPE) + .withUpdatedGeneration(repoName, safeGeneration, newGen) + ) + .build() + ) + .build(); } @Override public void onFailure(String source, Exception e) { listener.onFailure( - new RepositoryException(metadata.name(), "Failed to execute cluster state update [" + source + "]", e)); + new RepositoryException(metadata.name(), "Failed to execute cluster state update [" + source + "]", e) + ); } @Override @@ -1821,7 +2177,8 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS logger.trace("[{}] successfully set pending repository generation to [{}]", metadata.name(), newGen); setPendingStep.onResponse(newGen); } - }); + } + ); final StepListener filterRepositoryDataStep = new StepListener<>(); @@ -1830,7 +2187,9 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS // BwC logic: Load snapshot version information if any snapshot is missing details in RepositoryData so that the new // RepositoryData contains full details for every snapshot final List snapshotIdsWithMissingDetails = repositoryData.getSnapshotIds() - .stream().filter(repositoryData::hasMissingDetails).collect(Collectors.toList()); + .stream() + .filter(repositoryData::hasMissingDetails) + .collect(Collectors.toList()); if (snapshotIdsWithMissingDetails.isEmpty() == false) { final Map extraDetailsMap = new ConcurrentHashMap<>(); getSnapshotInfo( @@ -1838,29 +2197,33 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS snapshotIdsWithMissingDetails, false, () -> false, - (context, snapshotInfo) -> extraDetailsMap.put(snapshotInfo.snapshotId(), + (context, snapshotInfo) -> extraDetailsMap.put( + snapshotInfo.snapshotId(), new SnapshotDetails( snapshotInfo.state(), snapshotInfo.version(), snapshotInfo.startTime(), - snapshotInfo.endTime())), - ActionListener.runAfter( - new ActionListener<>() { - @Override - public void onResponse(Void aVoid) { - logger.info( - "Successfully loaded all snapshots' detailed information for {} from snapshot metadata", - AllocationService.firstListElementsToCommaDelimitedString( - snapshotIdsWithMissingDetails, SnapshotId::toString, logger.isDebugEnabled())); - } - - @Override - public void onFailure(Exception e) { - logger.warn("Failure when trying to load missing details from snapshot metadata", e); - } - }, - () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap)) + snapshotInfo.endTime() ) + ), + ActionListener.runAfter(new ActionListener<>() { + @Override + public void onResponse(Void aVoid) { + logger.info( + "Successfully loaded all snapshots' detailed information for {} from snapshot metadata", + AllocationService.firstListElementsToCommaDelimitedString( + snapshotIdsWithMissingDetails, + SnapshotId::toString, + logger.isDebugEnabled() + ) + ); + } + + @Override + public void onFailure(Exception e) { + logger.warn("Failure when trying to load missing details from snapshot metadata", e); + } + }, () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap))) ) ); } else { @@ -1872,8 +2235,12 @@ public void onFailure(Exception e) { final RepositoryData newRepositoryData = updateRepositoryData(filteredRepositoryData, version, newGen); if (latestKnownRepoGen.get() >= newGen) { throw new IllegalArgumentException( - "Tried writing generation [" + newGen + "] but repository is at least at generation [" + latestKnownRepoGen.get() - + "] already"); + "Tried writing generation [" + + newGen + + "] but repository is at least at generation [" + + latestKnownRepoGen.get() + + "] already" + ); } // write the index file if (ensureSafeGenerationExists(expectedGen, listener::onFailure) == false) { @@ -1891,34 +2258,44 @@ public void onFailure(Exception e) { maybeWriteIndexLatest(newGen); // Step 3: Update CS to reflect new repository generation. - clusterService.submitStateUpdateTask("set safe repository generation [" + metadata.name() + "][" + newGen + "]", + clusterService.submitStateUpdateTask( + "set safe repository generation [" + metadata.name() + "][" + newGen + "]", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { final RepositoryMetadata meta = getRepoMetadata(currentState); if (meta.generation() != expectedGen) { - throw new IllegalStateException("Tried to update repo generation to [" + newGen - + "] but saw unexpected generation in state [" + meta + "]"); + throw new IllegalStateException( + "Tried to update repo generation to [" + newGen + "] but saw unexpected generation in state [" + meta + "]" + ); } if (meta.pendingGeneration() != newGen) { throw new IllegalStateException( - "Tried to update from unexpected pending repo generation [" + meta.pendingGeneration() + - "] after write to generation [" + newGen + "]"); + "Tried to update from unexpected pending repo generation [" + + meta.pendingGeneration() + + "] after write to generation [" + + newGen + + "]" + ); } final RepositoriesMetadata currentMetadata = currentState.metadata().custom(RepositoriesMetadata.TYPE); final RepositoriesMetadata withGenerations = currentMetadata.withUpdatedGeneration(metadata.name(), newGen, newGen); final RepositoriesMetadata withUuid = meta.uuid().equals(newRepositoryData.getUuid()) - ? withGenerations - : withGenerations.withUuid(metadata.name(), newRepositoryData.getUuid()); - final ClusterState newClusterState = stateFilter.apply(ClusterState.builder(currentState).metadata( - Metadata.builder(currentState.getMetadata()).putCustom(RepositoriesMetadata.TYPE, withUuid)).build()); + ? withGenerations + : withGenerations.withUuid(metadata.name(), newRepositoryData.getUuid()); + final ClusterState newClusterState = stateFilter.apply( + ClusterState.builder(currentState) + .metadata(Metadata.builder(currentState.getMetadata()).putCustom(RepositoriesMetadata.TYPE, withUuid)) + .build() + ); return updateRepositoryGenerationsIfNecessary(newClusterState, expectedGen, newGen); } @Override public void onFailure(String source, Exception e) { listener.onFailure( - new RepositoryException(metadata.name(), "Failed to execute cluster state update [" + source + "]", e)); + new RepositoryException(metadata.name(), "Failed to execute cluster state update [" + source + "]", e) + ); } @Override @@ -1934,16 +2311,20 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS deleteFromContainer( blobContainer(), LongStream.range(Math.max(Math.max(expectedGen - 1, 0), newGen - 1000), newGen) - .mapToObj(gen -> INDEX_FILE_PREFIX + gen).iterator() + .mapToObj(gen -> INDEX_FILE_PREFIX + gen) + .iterator() ); } catch (IOException e) { - logger.warn(() -> - new ParameterizedMessage("Failed to clean up old index blobs from before [{}]", newGen), e); + logger.warn( + () -> new ParameterizedMessage("Failed to clean up old index blobs from before [{}]", newGen), + e + ); } return newRepositoryData; })); } - }); + } + ); }, listener::onFailure); } @@ -1968,9 +2349,15 @@ private void maybeWriteIndexLatest(long newGen) { try { writeAtomic(blobContainer(), INDEX_LATEST_BLOB, new BytesArray(Numbers.longToBytes(newGen)), false); } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("Failed to write index.latest blob. If you do not intend to use this " + - "repository as the basis for a URL repository you may turn off attempting to write the index.latest blob by " + - "setting repository setting [{}] to [false]", SUPPORT_URL_REPO.getKey()), e); + logger.warn( + () -> new ParameterizedMessage( + "Failed to write index.latest blob. If you do not intend to use this " + + "repository as the basis for a URL repository you may turn off attempting to write the index.latest blob by " + + "setting repository setting [{}] to [false]", + SUPPORT_URL_REPO.getKey() + ), + e + ); } } } @@ -1994,9 +2381,13 @@ private boolean ensureSafeGenerationExists(long safeGeneration, Consumer deletionEntries = new ArrayList<>(); - for (SnapshotDeletionsInProgress.Entry entry : - state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { + for (SnapshotDeletionsInProgress.Entry entry : state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY) + .getEntries()) { if (entry.repository().equals(repoName) && entry.repositoryStateId() == oldGen) { deletionEntries.add(entry.withRepoGen(newGen)); changedDeletions = true; @@ -2070,8 +2461,9 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state, } private RepositoryMetadata getRepoMetadata(ClusterState state) { - final RepositoryMetadata repositoryMetadata = - state.getMetadata().custom(RepositoriesMetadata.TYPE).repository(metadata.name()); + final RepositoryMetadata repositoryMetadata = state.getMetadata() + .custom(RepositoriesMetadata.TYPE) + .repository(metadata.name()); assert repositoryMetadata != null; return repositoryMetadata; } @@ -2088,13 +2480,13 @@ long latestIndexBlobId() throws IOException { try { // First, try listing all index-N blobs (there should only be two index-N blobs at any given // time in a repository if cleanup is happening properly) and pick the index-N blob with the - // highest N value - this will be the latest index blob for the repository. Note, we do this + // highest N value - this will be the latest index blob for the repository. Note, we do this // instead of directly reading the index.latest blob to get the current index-N blob because // index.latest is not written atomically and is not immutable - on every index-N change, - // we first delete the old index.latest and then write the new one. If the repository is not + // we first delete the old index.latest and then write the new one. If the repository is not // read-only, it is possible that we try deleting the index.latest blob while it is being read - // by some other operation (such as the get snapshots operation). In some file systems, it is - // illegal to delete a file while it is being read elsewhere (e.g. Windows). For read-only + // by some other operation (such as the get snapshots operation). In some file systems, it is + // illegal to delete a file while it is being read elsewhere (e.g. Windows). For read-only // repositories, we read for index.latest, both because listing blob prefixes is often unsupported // and because the index.latest blob will never be deleted and re-written. return listBlobsToGetLatestIndexId(); @@ -2136,10 +2528,9 @@ private long latestGeneration(Collection rootBlobs) { return latest; } - private void writeAtomic(BlobContainer container, final String blobName, final BytesReference bytesRef, - boolean failIfAlreadyExists) throws IOException { - logger.trace(() -> - new ParameterizedMessage("[{}] Writing [{}] to {} atomically", metadata.name(), blobName, container.path())); + private void writeAtomic(BlobContainer container, final String blobName, final BytesReference bytesRef, boolean failIfAlreadyExists) + throws IOException { + logger.trace(() -> new ParameterizedMessage("[{}] Writing [{}] to {} atomically", metadata.name(), blobName, container.path())); container.writeBlobAtomic(blobName, bytesRef, failIfAlreadyExists); } @@ -2175,19 +2566,22 @@ public void snapshotShard(SnapshotShardContext context) { String fileListGeneration = tuple.v2(); if (snapshots.snapshots().stream().anyMatch(sf -> sf.snapshot().equals(snapshotId.getName()))) { - throw new IndexShardSnapshotFailedException(shardId, - "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting"); + throw new IndexShardSnapshotFailedException( + shardId, + "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting" + ); } // First inspect all known SegmentInfos instances to see if we already have an equivalent commit in the repository - final List filesFromSegmentInfos = - Optional.ofNullable(context.stateIdentifier()).map(id -> { - for (SnapshotFiles snapshotFileSet : snapshots.snapshots()) { - if (id.equals(snapshotFileSet.shardStateIdentifier())) { - return snapshotFileSet.indexFiles(); + final List filesFromSegmentInfos = Optional.ofNullable(context.stateIdentifier()) + .map(id -> { + for (SnapshotFiles snapshotFileSet : snapshots.snapshots()) { + if (id.equals(snapshotFileSet.shardStateIdentifier())) { + return snapshotFileSet.indexFiles(); + } } - } - return null; - }).orElse(null); + return null; + }) + .orElse(null); final List indexCommitPointFiles; int indexIncrementalFileCount = 0; @@ -2204,8 +2598,7 @@ public void snapshotShard(SnapshotShardContext context) { try (Releasable ignored = incrementStoreRef(store, snapshotStatus, shardId)) { // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should try { - logger.trace( - "[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); + logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); metadataFromStore = store.getMetadata(snapshotIndexCommit); fileNames = snapshotIndexCommit.getFileNames(); } catch (IOException e) { @@ -2243,10 +2636,11 @@ public void snapshotShard(SnapshotShardContext context) { indexIncrementalFileCount++; indexIncrementalSize += md.length(); // create a new FileInfo - BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = - new BlobStoreIndexShardSnapshot.FileInfo( - (needsWrite ? UPLOADED_DATA_BLOB_PREFIX : VIRTUAL_DATA_BLOB_PREFIX) + UUIDs.randomBase64UUID(), - md, chunkSize()); + BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = new BlobStoreIndexShardSnapshot.FileInfo( + (needsWrite ? UPLOADED_DATA_BLOB_PREFIX : VIRTUAL_DATA_BLOB_PREFIX) + UUIDs.randomBase64UUID(), + md, + chunkSize() + ); indexCommitPointFiles.add(snapshotFileInfo); if (needsWrite) { filesToSnapshot.add(snapshotFileInfo); @@ -2264,8 +2658,13 @@ public void snapshotShard(SnapshotShardContext context) { indexCommitPointFiles = filesFromSegmentInfos; } - snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount, - indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileSize); + snapshotStatus.moveToStarted( + startTime, + indexIncrementalFileCount, + indexTotalNumberOfFiles, + indexIncrementalSize, + indexTotalFileSize + ); final String indexGeneration; final boolean writeShardGens = SnapshotsService.useShardGenerations(context.getRepositoryMetaVersion()); @@ -2283,12 +2682,23 @@ public void snapshotShard(SnapshotShardContext context) { // reference a generation that has not had all its files fully upload. indexGeneration = UUIDs.randomBase64UUID(); try { - INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedBlobStoreIndexShardSnapshots, shardContainer, indexGeneration, compress, - bigArrays); + INDEX_SHARD_SNAPSHOTS_FORMAT.write( + updatedBlobStoreIndexShardSnapshots, + shardContainer, + indexGeneration, + compress, + bigArrays + ); } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, - "Failed to write shard level snapshot metadata for [" + snapshotId + "] to [" - + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(indexGeneration) + "]", e); + throw new IndexShardSnapshotFailedException( + shardId, + "Failed to write shard level snapshot metadata for [" + + snapshotId + + "] to [" + + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(indexGeneration) + + "]", + e + ); } afterWriteSnapBlob = () -> {}; } else { @@ -2298,44 +2708,60 @@ public void snapshotShard(SnapshotShardContext context) { final long newGen = Long.parseLong(fileListGeneration) + 1; indexGeneration = Long.toString(newGen); // Delete all previous index-N blobs - final List blobsToDelete = blobs.stream().filter(blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)) - .collect(Collectors.toList()); - assert blobsToDelete.stream().mapToLong(b -> Long.parseLong(b.replaceFirst(SNAPSHOT_INDEX_PREFIX, ""))) - .max().orElse(-1L) < Long.parseLong(indexGeneration) - : "Tried to delete an index-N blob newer than the current generation [" + indexGeneration - + "] when deleting index-N blobs " + blobsToDelete; + final List blobsToDelete = blobs.stream() + .filter(blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)) + .collect(Collectors.toList()); + assert blobsToDelete.stream() + .mapToLong(b -> Long.parseLong(b.replaceFirst(SNAPSHOT_INDEX_PREFIX, ""))) + .max() + .orElse(-1L) < Long.parseLong(indexGeneration) + : "Tried to delete an index-N blob newer than the current generation [" + + indexGeneration + + "] when deleting index-N blobs " + + blobsToDelete; afterWriteSnapBlob = () -> { try { writeShardIndexBlobAtomic(shardContainer, newGen, updatedBlobStoreIndexShardSnapshots); } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, - "Failed to finalize snapshot creation [" + snapshotId + "] with shard index [" - + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(indexGeneration) + "]", e); + throw new IndexShardSnapshotFailedException( + shardId, + "Failed to finalize snapshot creation [" + + snapshotId + + "] with shard index [" + + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(indexGeneration) + + "]", + e + ); } try { deleteFromContainer(shardContainer, blobsToDelete.iterator()); } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete old index-N blobs during finalization", - snapshotId, shardId), e); + logger.warn( + () -> new ParameterizedMessage( + "[{}][{}] failed to delete old index-N blobs during finalization", + snapshotId, + shardId + ), + e + ); } }; } final StepListener> allFilesUploadedListener = new StepListener<>(); allFilesUploadedListener.whenComplete(v -> { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = - snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); // now create and write the commit point logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); final BlobStoreIndexShardSnapshot blobStoreIndexShardSnapshot = new BlobStoreIndexShardSnapshot( - snapshotId.getName(), - lastSnapshotStatus.getIndexVersion(), - indexCommitPointFiles, - lastSnapshotStatus.getStartTime(), - threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(), - lastSnapshotStatus.getIncrementalFileCount(), - lastSnapshotStatus.getIncrementalSize() + snapshotId.getName(), + lastSnapshotStatus.getIndexVersion(), + indexCommitPointFiles, + lastSnapshotStatus.getStartTime(), + threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(), + lastSnapshotStatus.getIncrementalFileCount(), + lastSnapshotStatus.getIncrementalSize() ); try { final String snapshotUUID = snapshotId.getUUID(); @@ -2345,9 +2771,10 @@ public void snapshotShard(SnapshotShardContext context) { } afterWriteSnapBlob.run(); final ShardSnapshotResult shardSnapshotResult = new ShardSnapshotResult( - indexGeneration, - ByteSizeValue.ofBytes(blobStoreIndexShardSnapshot.totalSize()), - snapshotIndexCommit.getSegmentCount()); + indexGeneration, + ByteSizeValue.ofBytes(blobStoreIndexShardSnapshot.totalSize()), + snapshotIndexCommit.getSegmentCount() + ); snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis(), shardSnapshotResult); context.onResponse(shardSnapshotResult); }, context::onFailure); @@ -2367,9 +2794,15 @@ public void snapshotShard(SnapshotShardContext context) { } } - private void executeOneFileSnapshot(Store store, SnapshotId snapshotId, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, - BlockingQueue filesToSnapshot, Executor executor, - ActionListener listener) throws InterruptedException { + private void executeOneFileSnapshot( + Store store, + SnapshotId snapshotId, + IndexId indexId, + IndexShardSnapshotStatus snapshotStatus, + BlockingQueue filesToSnapshot, + Executor executor, + ActionListener listener + ) throws InterruptedException { final ShardId shardId = store.shardId(); final BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = filesToSnapshot.poll(0L, TimeUnit.MILLISECONDS); if (snapshotFileInfo == null) { @@ -2396,8 +2829,11 @@ private static Releasable incrementStoreRef(Store store, IndexShardSnapshotStatu return store::decRef; } - private static boolean assertFileContentsMatchHash(IndexShardSnapshotStatus snapshotStatus, - BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store store) { + private static boolean assertFileContentsMatchHash( + IndexShardSnapshotStatus snapshotStatus, + BlobStoreIndexShardSnapshot.FileInfo fileInfo, + Store store + ) { if (store.tryIncRef()) { try (IndexInput indexInput = store.openVerifyingInput(fileInfo.physicalName(), IOContext.READONCE, fileInfo.metadata())) { final byte[] tmp = new byte[Math.toIntExact(fileInfo.metadata().length())]; @@ -2415,11 +2851,18 @@ private static boolean assertFileContentsMatchHash(IndexShardSnapshotStatus snap } @Override - public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, - RecoveryState recoveryState, ActionListener listener) { + public void restoreShard( + Store store, + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId, + RecoveryState recoveryState, + ActionListener listener + ) { final ShardId shardId = store.shardId(); final ActionListener restoreListener = listener.delegateResponse( - (l, e) -> l.onFailure(new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e))); + (l, e) -> l.onFailure(new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e)) + ); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); final BlobContainer container = shardContainer(indexId, snapshotShardId); executor.execute(ActionRunnable.wrap(restoreListener, l -> { @@ -2427,14 +2870,19 @@ public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, Sh final SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles(), null); new FileRestoreContext(metadata.name(), shardId, snapshotId, recoveryState) { @Override - protected void restoreFiles(List filesToRecover, Store store, - ActionListener listener) { + protected void restoreFiles( + List filesToRecover, + Store store, + ActionListener listener + ) { if (filesToRecover.isEmpty()) { listener.onResponse(null); } else { // Start as many workers as fit into the snapshot pool at once at the most - final int workers = - Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), snapshotFiles.indexFiles().size()); + final int workers = Math.min( + threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), + snapshotFiles.indexFiles().size() + ); final BlockingQueue files = new LinkedBlockingQueue<>(filesToRecover); final ActionListener allFilesListener = fileQueueListener(files, workers, listener.map(v -> null)); // restore the files from the snapshot to the Lucene store @@ -2448,8 +2896,10 @@ protected void restoreFiles(List filesToRe } } - private void executeOneFileRestore(BlockingQueue files, - ActionListener allFilesListener) throws InterruptedException { + private void executeOneFileRestore( + BlockingQueue files, + ActionListener allFilesListener + ) throws InterruptedException { final BlobStoreIndexShardSnapshot.FileInfo fileToRecover = files.poll(0L, TimeUnit.MILLISECONDS); if (fileToRecover == null) { allFilesListener.onResponse(null); @@ -2470,8 +2920,13 @@ private void restoreFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store st ensureNotClosing(store); logger.trace(() -> new ParameterizedMessage("[{}] restoring [{}] to [{}]", metadata.name(), fileInfo, store)); boolean success = false; - try (IndexOutput indexOutput = - store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { + try ( + IndexOutput indexOutput = store.createVerifyingOutput( + fileInfo.physicalName(), + fileInfo.metadata(), + IOContext.DEFAULT + ) + ) { if (fileInfo.name().startsWith(VIRTUAL_DATA_BLOB_PREFIX)) { final BytesRef hash = fileInfo.metadata().hash(); indexOutput.writeBytes(hash.bytes, hash.offset, hash.length); @@ -2522,8 +2977,11 @@ void ensureNotClosing(final Store store) throws AlreadyClosedException { })); } - private static ActionListener fileQueueListener(BlockingQueue files, int workers, - ActionListener> listener) { + private static ActionListener fileQueueListener( + BlockingQueue files, + int workers, + ActionListener> listener + ) { return new GroupedActionListener<>(listener, workers).delegateResponse((l, e) -> { files.clear(); // Stop uploading the remaining files if we run into any exception l.onFailure(e); @@ -2531,9 +2989,10 @@ private static ActionListener fileQueueListener(BlockingQueue rateLimiterSupplier, - RateLimitingInputStream.Listener throttleListener) { + InputStream stream, + Supplier rateLimiterSupplier, + RateLimitingInputStream.Listener throttleListener + ) { return new RateLimitingInputStream(stream, rateLimiterSupplier, throttleListener); } @@ -2552,8 +3011,11 @@ public InputStream maybeRateLimitRestores(InputStream stream) { * BlobStoreRepository#getRestoreThrottleTimeInNanos}. */ public InputStream maybeRateLimitRestores(InputStream stream, RateLimitingInputStream.Listener throttleListener) { - return maybeRateLimit(maybeRateLimit(stream, () -> restoreRateLimiter, throttleListener), - recoverySettings::rateLimiter, throttleListener); + return maybeRateLimit( + maybeRateLimit(stream, () -> restoreRateLimiter, throttleListener), + recoverySettings::rateLimiter, + throttleListener + ); } /** @@ -2576,9 +3038,15 @@ public InputStream maybeRateLimitSnapshots(InputStream stream, RateLimitingInput @Override public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) { BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotId); - return IndexShardSnapshotStatus.newDone(snapshot.startTime(), snapshot.time(), - snapshot.incrementalFileCount(), snapshot.totalFileCount(), - snapshot.incrementalSize(), snapshot.totalSize(), null); // Not adding a real generation here as it doesn't matter to callers + return IndexShardSnapshotStatus.newDone( + snapshot.startTime(), + snapshot.time(), + snapshot.incrementalFileCount(), + snapshot.totalFileCount(), + snapshot.incrementalSize(), + snapshot.totalSize(), + null + ); // Not adding a real generation here as it doesn't matter to callers } @Override @@ -2588,28 +3056,45 @@ public void verify(String seed, DiscoveryNode localNode) { try { latestIndexBlobId(); } catch (Exception e) { - throw new RepositoryVerificationException(metadata.name(), "path " + basePath() + - " is not accessible on node " + localNode, e); + throw new RepositoryVerificationException( + metadata.name(), + "path " + basePath() + " is not accessible on node " + localNode, + e + ); } } else { BlobContainer testBlobContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); try { testBlobContainer.writeBlob("data-" + localNode.getId() + ".dat", new BytesArray(seed), true); } catch (Exception exp) { - throw new RepositoryVerificationException(metadata.name(), "store location [" + blobStore() + - "] is not accessible on the node [" + localNode + "]", exp); + throw new RepositoryVerificationException( + metadata.name(), + "store location [" + blobStore() + "] is not accessible on the node [" + localNode + "]", + exp + ); } try (InputStream masterDat = testBlobContainer.readBlob("master.dat")) { final String seedRead = Streams.readFully(masterDat).utf8ToString(); if (seedRead.equals(seed) == false) { - throw new RepositoryVerificationException(metadata.name(), "Seed read from master.dat was [" + seedRead + - "] but expected seed [" + seed + "]"); + throw new RepositoryVerificationException( + metadata.name(), + "Seed read from master.dat was [" + seedRead + "] but expected seed [" + seed + "]" + ); } } catch (NoSuchFileException e) { - throw new RepositoryVerificationException(metadata.name(), "a file written by master to the store [" + blobStore() + - "] cannot be accessed on the node [" + localNode + "]. " + - "This might indicate that the store [" + blobStore() + "] is not shared between this node and the master node or " + - "that permissions on the store don't allow reading files written by the master node", e); + throw new RepositoryVerificationException( + metadata.name(), + "a file written by master to the store [" + + blobStore() + + "] cannot be accessed on the node [" + + localNode + + "]. " + + "This might indicate that the store [" + + blobStore() + + "] is not shared between this node and the master node or " + + "that permissions on the store don't allow reading files written by the master node", + e + ); } catch (Exception e) { throw new RepositoryVerificationException(metadata.name(), "Failed to verify repository", e); } @@ -2618,10 +3103,7 @@ public void verify(String seed, DiscoveryNode localNode) { @Override public String toString() { - return "BlobStoreRepository[" + - "[" + metadata.name() + - "], [" + blobStore.get() + ']' + - ']'; + return "BlobStoreRepository[" + "[" + metadata.name() + "], [" + blobStore.get() + ']' + ']'; } /** @@ -2630,11 +3112,16 @@ public String toString() { * @param indexGeneration generation to write the new shard level level metadata to. If negative a uuid id shard generation should be * used */ - private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta(Set survivingSnapshots, IndexId indexId, - int snapshotShardId, Collection snapshotIds, - BlobContainer shardContainer, Set blobs, - BlobStoreIndexShardSnapshots snapshots, - long indexGeneration) { + private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta( + Set survivingSnapshots, + IndexId indexId, + int snapshotShardId, + Collection snapshotIds, + BlobContainer shardContainer, + Set blobs, + BlobStoreIndexShardSnapshots snapshots, + long indexGeneration + ) { // Build a list of snapshots that should be preserved List newSnapshotsList = new ArrayList<>(); final Set survivingSnapshotNames = survivingSnapshots.stream().map(SnapshotId::getName).collect(Collectors.toSet()); @@ -2657,12 +3144,23 @@ private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta(Set survivingSnapshotUUIDs = survivingSnapshots.stream().map(SnapshotId::getUUID).collect(Collectors.toSet()); - return new ShardSnapshotMetaDeleteResult(indexId, snapshotShardId, writtenGeneration, - unusedBlobs(blobs, survivingSnapshotUUIDs, updatedSnapshots)); + return new ShardSnapshotMetaDeleteResult( + indexId, + snapshotShardId, + writtenGeneration, + unusedBlobs(blobs, survivingSnapshotUUIDs, updatedSnapshots) + ); } } catch (IOException e) { - throw new RepositoryException(metadata.name(), "Failed to finalize snapshot deletion " + snapshotIds + - " with shard index [" + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(writtenGeneration) + "]", e); + throw new RepositoryException( + metadata.name(), + "Failed to finalize snapshot deletion " + + snapshotIds + + " with shard index [" + + INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(writtenGeneration) + + "]", + e + ); } } @@ -2670,27 +3168,44 @@ private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta(Set= 0 : "Shard generation must not be negative but saw [" + indexGeneration + "]"; - logger.trace(() -> new ParameterizedMessage("[{}] Writing shard index [{}] to [{}]", metadata.name(), - indexGeneration, shardContainer.path())); + logger.trace( + () -> new ParameterizedMessage("[{}] Writing shard index [{}] to [{}]", metadata.name(), indexGeneration, shardContainer.path()) + ); final String blobName = INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(String.valueOf(indexGeneration)); - INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress, bigArrays, - bytes -> writeAtomic(shardContainer, blobName, bytes, true)); + INDEX_SHARD_SNAPSHOTS_FORMAT.serialize( + updatedSnapshots, + blobName, + compress, + bigArrays, + bytes -> writeAtomic(shardContainer, blobName, bytes, true) + ); } // Unused blobs are all previous index-, data- and meta-blobs and that are not referenced by the new index- as well as all // temporary blobs - private static List unusedBlobs(Set blobs, Set survivingSnapshotUUIDs, - BlobStoreIndexShardSnapshots updatedSnapshots) { - return blobs.stream().filter(blob -> - blob.startsWith(SNAPSHOT_INDEX_PREFIX) - || (blob.startsWith(SNAPSHOT_PREFIX) && blob.endsWith(".dat") - && survivingSnapshotUUIDs.contains( - blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length())) == false) - || (blob.startsWith(UPLOADED_DATA_BLOB_PREFIX) && updatedSnapshots.findNameFile(canonicalName(blob)) == null) - || FsBlobContainer.isTempBlobName(blob)).collect(Collectors.toList()); + private static List unusedBlobs( + Set blobs, + Set survivingSnapshotUUIDs, + BlobStoreIndexShardSnapshots updatedSnapshots + ) { + return blobs.stream() + .filter( + blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX) + || (blob.startsWith(SNAPSHOT_PREFIX) + && blob.endsWith(".dat") + && survivingSnapshotUUIDs.contains( + blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - ".dat".length()) + ) == false) + || (blob.startsWith(UPLOADED_DATA_BLOB_PREFIX) && updatedSnapshots.findNameFile(canonicalName(blob)) == null) + || FsBlobContainer.isTempBlobName(blob) + ) + .collect(Collectors.toList()); } /** @@ -2702,8 +3217,12 @@ public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContaine } catch (NoSuchFileException ex) { throw new SnapshotMissingException(metadata.name(), snapshotId, ex); } catch (IOException ex) { - throw new SnapshotException(metadata.name(), snapshotId, - "failed to read shard snapshot file for [" + shardContainer.path() + ']', ex); + throw new SnapshotException( + metadata.name(), + snapshotId, + "failed to read shard snapshot file for [" + shardContainer.path() + ']', + ex + ); } } @@ -2717,9 +3236,11 @@ public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContaine * {@link SnapshotsService#SHARD_GEN_IN_REPO_DATA_VERSION}. * @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation */ - private Tuple buildBlobStoreIndexShardSnapshots(Set blobs, - BlobContainer shardContainer, - @Nullable String generation) throws IOException { + private Tuple buildBlobStoreIndexShardSnapshots( + Set blobs, + BlobContainer shardContainer, + @Nullable String generation + ) throws IOException { if (generation != null) { if (generation.equals(ShardGenerations.NEW_SHARD_GEN)) { return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, ShardGenerations.NEW_SHARD_GEN); @@ -2737,16 +3258,21 @@ private Tuple buildBlobStoreIndexShardSnap * @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation */ private Tuple buildBlobStoreIndexShardSnapshots(Set blobs, BlobContainer shardContainer) - throws IOException { + throws IOException { long latest = latestGeneration(blobs); if (latest >= 0) { - final BlobStoreIndexShardSnapshots shardSnapshots = - INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, Long.toString(latest), namedXContentRegistry); + final BlobStoreIndexShardSnapshots shardSnapshots = INDEX_SHARD_SNAPSHOTS_FORMAT.read( + shardContainer, + Long.toString(latest), + namedXContentRegistry + ); return new Tuple<>(shardSnapshots, latest); - } else if (blobs.stream().anyMatch(b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(INDEX_FILE_PREFIX) - || b.startsWith(UPLOADED_DATA_BLOB_PREFIX))) { - logger.warn("Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]"); - } + } else if (blobs.stream() + .anyMatch(b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(INDEX_FILE_PREFIX) || b.startsWith(UPLOADED_DATA_BLOB_PREFIX))) { + logger.warn( + "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]" + ); + } return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, latest); } @@ -2754,8 +3280,14 @@ private Tuple buildBlobStoreIndexShardSnapsh * Snapshot individual file * @param fileInfo file to be snapshotted */ - private void snapshotFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, IndexId indexId, ShardId shardId, SnapshotId snapshotId, - IndexShardSnapshotStatus snapshotStatus, Store store) throws IOException { + private void snapshotFile( + BlobStoreIndexShardSnapshot.FileInfo fileInfo, + IndexId indexId, + ShardId shardId, + SnapshotId snapshotId, + IndexShardSnapshotStatus snapshotStatus, + Store store + ) throws IOException { final BlobContainer shardContainer = shardContainer(indexId, shardId); final String file = fileInfo.physicalName(); try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) { @@ -2763,8 +3295,9 @@ private void snapshotFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, IndexId final long partBytes = fileInfo.partBytes(i); // Make reads abortable by mutating the snapshotStatus object - final InputStream inputStream = new FilterInputStream(maybeRateLimitSnapshots( - new InputStreamIndexInput(indexInput, partBytes))) { + final InputStream inputStream = new FilterInputStream( + maybeRateLimitSnapshots(new InputStreamIndexInput(indexInput, partBytes)) + ) { @Override public int read() throws IOException { checkAborted(); @@ -2779,15 +3312,13 @@ public int read(byte[] b, int off, int len) throws IOException { private void checkAborted() { if (snapshotStatus.isAborted()) { - logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, - snapshotId, fileInfo.physicalName()); + logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileInfo.physicalName()); throw new AbortedSnapshotException(); } } }; final String partName = fileInfo.partName(i); - logger.trace(() -> - new ParameterizedMessage("[{}] Writing [{}] to [{}]", metadata.name(), partName, shardContainer.path())); + logger.trace(() -> new ParameterizedMessage("[{}] Writing [{}] to [{}]", metadata.name(), partName, shardContainer.path())); shardContainer.writeBlob(partName, inputStream, partBytes, false); } Store.verify(indexInput); diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index 1ceaf6a74a2d1..7a279b40850b9 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -114,8 +114,10 @@ public T deserialize(NamedXContentRegistry namedXContentRegistry, InputStream in wrappedStream = deserializeMetaBlobInputStream; } final T result; - try (XContentParser parser = XContentType.SMILE.xContent().createParser( - namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, wrappedStream)) { + try ( + XContentParser parser = XContentType.SMILE.xContent() + .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, wrappedStream) + ) { result = reader.apply(parser); } deserializeMetaBlobInputStream.verifyFooter(); @@ -213,8 +215,12 @@ private int doRead(byte[] b, int off, int len) throws IOException { void verifyFooter() throws CorruptStateException { if (bufferCount - bufferPos != CodecUtil.footerLength()) { throw new CorruptStateException( - "should have consumed all but 16 bytes from the buffer but saw buffer pos [" + bufferPos + "] and count [" - + bufferCount + "]"); + "should have consumed all but 16 bytes from the buffer but saw buffer pos [" + + bufferPos + + "] and count [" + + bufferCount + + "]" + ); } crc32.update(buffer, 0, bufferPos + 8); final int magicFound = Numbers.bytesToInt(buffer, bufferPos); @@ -278,12 +284,22 @@ public void write(T obj, BlobContainer blobContainer, String name, boolean compr serialize(obj, blobName, compress, bigArrays, bytes -> blobContainer.writeBlob(blobName, bytes, false)); } - public void serialize(final T obj, final String blobName, final boolean compress, BigArrays bigArrays, - CheckedConsumer consumer) throws IOException { + public void serialize( + final T obj, + final String blobName, + final boolean compress, + BigArrays bigArrays, + CheckedConsumer consumer + ) throws IOException { try (ReleasableBytesStreamOutput outputStream = new ReleasableBytesStreamOutput(bigArrays)) { - try (OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( - "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", blobName, - org.elasticsearch.common.io.Streams.noCloseStream(outputStream), BUFFER_SIZE)) { + try ( + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( + "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", + blobName, + org.elasticsearch.common.io.Streams.noCloseStream(outputStream), + BUFFER_SIZE + ) + ) { CodecUtil.writeHeader(indexOutput, codec, VERSION); try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { @Override @@ -291,9 +307,12 @@ public void close() { // this is important since some of the XContentBuilders write bytes on close. // in order to write the footer we need to prevent closing the actual index input. } - }; XContentBuilder builder = XContentFactory.contentBuilder(XContentType.SMILE, - compress ? CompressorFactory.COMPRESSOR.threadLocalOutputStream(indexOutputOutputStream) - : indexOutputOutputStream)) { + }; + XContentBuilder builder = XContentFactory.contentBuilder( + XContentType.SMILE, + compress ? CompressorFactory.COMPRESSOR.threadLocalOutputStream(indexOutputOutputStream) : indexOutputOutputStream + ) + ) { builder.startObject(); obj.toXContent(builder, SNAPSHOT_ONLY_FORMAT_PARAMS); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java index d2be6818b40ef..35624691f32dd 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java @@ -81,8 +81,14 @@ public void restore(SnapshotFiles snapshotFiles, Store store, ActionListener filesToRecover = new ArrayList<>(); @@ -105,8 +111,13 @@ public void restore(SnapshotFiles snapshotFiles, Store store, ActionListener { - store.incRef(); - try { - afterRestore(snapshotFiles, store, restoredSegmentsFile); - listener.onResponse(null); - } finally { - store.decRef(); - } - }, listener::onFailure)); + restoreFiles(filesToRecover, store, ActionListener.wrap(v -> { + store.incRef(); + try { + afterRestore(snapshotFiles, store, restoredSegmentsFile); + listener.onResponse(null); + } finally { + store.decRef(); + } + }, listener::onFailure)); } catch (IOException ex) { throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); } @@ -171,15 +180,18 @@ private void afterRestore(SnapshotFiles snapshotFiles, Store store, StoreFileMet Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); } } catch (IOException e) { - throw new IndexShardRestoreFailedException(shardId, "Failed to remove files not referenced in segment file [" - + restoredSegmentsFile.name() + "] after restore", e); + throw new IndexShardRestoreFailedException( + shardId, + "Failed to remove files not referenced in segment file [" + restoredSegmentsFile.name() + "] after restore", + e + ); } /// now, go over and clean files that are in the store, but were not in the snapshot try { for (String storeFile : store.directory().listAll()) { if (Store.isAutogenerated(storeFile) || snapshotFiles.containPhysicalIndexFile(storeFile)) { - continue; //skip write.lock, checksum files and files that exist in the snapshot + continue; // skip write.lock, checksum files and files that exist in the snapshot } try { store.deleteQuiet("restore", storeFile); @@ -203,8 +215,11 @@ private void afterRestore(SnapshotFiles snapshotFiles, Store store, StoreFileMet * @param filesToRecover List of files to restore * @param store Store to restore into */ - protected abstract void restoreFiles(List filesToRecover, Store store, - ActionListener listener); + protected abstract void restoreFiles( + List filesToRecover, + Store store, + ActionListener listener + ); @SuppressWarnings("unchecked") private static Iterable concat(Store.RecoveryDiff diff) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MeteredBlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MeteredBlobStoreRepository.java index 079dd85cc3a6f..db29862688433 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MeteredBlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MeteredBlobStoreRepository.java @@ -24,20 +24,24 @@ public abstract class MeteredBlobStoreRepository extends BlobStoreRepository { private final RepositoryInfo repositoryInfo; - public MeteredBlobStoreRepository(RepositoryMetadata metadata, - NamedXContentRegistry namedXContentRegistry, - ClusterService clusterService, - BigArrays bigArrays, - RecoverySettings recoverySettings, - BlobPath basePath, - Map location) { + public MeteredBlobStoreRepository( + RepositoryMetadata metadata, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings, + BlobPath basePath, + Map location + ) { super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, basePath); ThreadPool threadPool = clusterService.getClusterApplierService().threadPool(); - this.repositoryInfo = new RepositoryInfo(UUIDs.randomBase64UUID(), + this.repositoryInfo = new RepositoryInfo( + UUIDs.randomBase64UUID(), metadata.name(), metadata.type(), location, - threadPool.absoluteTimeInMillis()); + threadPool.absoluteTimeInMillis() + ); } public RepositoryStatsSnapshot statsSnapshot() { diff --git a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java index 5f812cc93e06f..3b7449aa4bfd7 100644 --- a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java @@ -45,14 +45,27 @@ public class FsRepository extends BlobStoreRepository { public static final String TYPE = "fs"; - public static final Setting LOCATION_SETTING = - new Setting<>("location", "", Function.identity(), Property.NodeScope); - public static final Setting REPOSITORIES_LOCATION_SETTING = - new Setting<>("repositories.fs.location", LOCATION_SETTING, Function.identity(), Property.NodeScope); - public static final Setting CHUNK_SIZE_SETTING = Setting.byteSizeSetting("chunk_size", - new ByteSizeValue(Long.MAX_VALUE), new ByteSizeValue(5), new ByteSizeValue(Long.MAX_VALUE), Property.NodeScope); - public static final Setting REPOSITORIES_CHUNK_SIZE_SETTING = Setting.byteSizeSetting("repositories.fs.chunk_size", - new ByteSizeValue(Long.MAX_VALUE), new ByteSizeValue(5), new ByteSizeValue(Long.MAX_VALUE), Property.NodeScope); + public static final Setting LOCATION_SETTING = new Setting<>("location", "", Function.identity(), Property.NodeScope); + public static final Setting REPOSITORIES_LOCATION_SETTING = new Setting<>( + "repositories.fs.location", + LOCATION_SETTING, + Function.identity(), + Property.NodeScope + ); + public static final Setting CHUNK_SIZE_SETTING = Setting.byteSizeSetting( + "chunk_size", + new ByteSizeValue(Long.MAX_VALUE), + new ByteSizeValue(5), + new ByteSizeValue(Long.MAX_VALUE), + Property.NodeScope + ); + public static final Setting REPOSITORIES_CHUNK_SIZE_SETTING = Setting.byteSizeSetting( + "repositories.fs.chunk_size", + new ByteSizeValue(Long.MAX_VALUE), + new ByteSizeValue(5), + new ByteSizeValue(Long.MAX_VALUE), + Property.NodeScope + ); private final Environment environment; private final ByteSizeValue chunkSize; @@ -60,28 +73,46 @@ public class FsRepository extends BlobStoreRepository { /** * Constructs a shared file system repository. */ - public FsRepository(RepositoryMetadata metadata, Environment environment, NamedXContentRegistry namedXContentRegistry, - ClusterService clusterService, BigArrays bigArrays, RecoverySettings recoverySettings) { + public FsRepository( + RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings + ) { super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, BlobPath.EMPTY); this.environment = environment; String location = REPOSITORIES_LOCATION_SETTING.get(metadata.settings()); if (location.isEmpty()) { - logger.warn("the repository location is missing, it should point to a shared file system location" - + " that is available on all master and data nodes"); + logger.warn( + "the repository location is missing, it should point to a shared file system location" + + " that is available on all master and data nodes" + ); throw new RepositoryException(metadata.name(), "missing location"); } Path locationFile = environment.resolveRepoFile(location); if (locationFile == null) { if (environment.repoFiles().length > 0) { - logger.warn("The specified location [{}] doesn't start with any " - + "repository paths specified by the path.repo setting: [{}] ", location, environment.repoFiles()); - throw new RepositoryException(metadata.name(), "location [" + location - + "] doesn't match any of the locations specified by path.repo"); + logger.warn( + "The specified location [{}] doesn't start with any " + "repository paths specified by the path.repo setting: [{}] ", + location, + environment.repoFiles() + ); + throw new RepositoryException( + metadata.name(), + "location [" + location + "] doesn't match any of the locations specified by path.repo" + ); } else { - logger.warn("The specified location [{}] should start with a repository path specified by" - + " the path.repo setting, but the path.repo setting was not set on this node", location); - throw new RepositoryException(metadata.name(), "location [" + location - + "] doesn't match any of the locations specified by path.repo because this setting is empty"); + logger.warn( + "The specified location [{}] should start with a repository path specified by" + + " the path.repo setting, but the path.repo setting was not set on this node", + location + ); + throw new RepositoryException( + metadata.name(), + "location [" + location + "] doesn't match any of the locations specified by path.repo because this setting is empty" + ); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java b/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java index f24147dd7fcd8..0af62b9944dba 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java +++ b/server/src/main/java/org/elasticsearch/snapshots/InFlightShardSnapshotStates.java @@ -61,14 +61,19 @@ public static InFlightShardSnapshotStates forRepo(String repoName, List> generations, Map> busyIds, - SnapshotsInProgress.ShardSnapshotStatus shardState, int shardId, String indexName) { + private static void addStateInformation( + Map> generations, + Map> busyIds, + SnapshotsInProgress.ShardSnapshotStatus shardState, + int shardId, + String indexName + ) { if (shardState.isActive()) { busyIds.computeIfAbsent(indexName, k -> new HashSet<>()).add(shardId); assert assertGenerationConsistency(generations, indexName, shardId, shardState.generation()); } else if (shardState.state() == SnapshotsInProgress.ShardState.SUCCESS) { - assert busyIds.getOrDefault(indexName, Collections.emptySet()).contains(shardId) == false : - "Can't have a successful operation queued after an in-progress operation"; + assert busyIds.getOrDefault(indexName, Collections.emptySet()).contains(shardId) == false + : "Can't have a successful operation queued after an in-progress operation"; generations.computeIfAbsent(indexName, k -> new HashMap<>()).put(shardId, shardState.generation()); } } @@ -84,14 +89,17 @@ private static void addStateInformation(Map> genera */ private final Map> activeShardIds; - private InFlightShardSnapshotStates(Map> generations, Map> activeShardIds) { this.generations = generations; this.activeShardIds = activeShardIds; } - private static boolean assertGenerationConsistency(Map> generations, String indexName, - int shardId, @Nullable String activeGeneration) { + private static boolean assertGenerationConsistency( + Map> generations, + String indexName, + int shardId, + @Nullable String activeGeneration + ) { final String bestGeneration = generations.getOrDefault(indexName, Collections.emptyMap()).get(shardId); assert bestGeneration == null || activeGeneration == null || activeGeneration.equals(bestGeneration); return true; diff --git a/server/src/main/java/org/elasticsearch/snapshots/InternalSnapshotsInfoService.java b/server/src/main/java/org/elasticsearch/snapshots/InternalSnapshotsInfoService.java index 7b0073c9c6ff3..a152febe19b36 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/InternalSnapshotsInfoService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/InternalSnapshotsInfoService.java @@ -45,9 +45,13 @@ public class InternalSnapshotsInfoService implements ClusterStateListener, SnapshotsInfoService { - public static final Setting INTERNAL_SNAPSHOT_INFO_MAX_CONCURRENT_FETCHES_SETTING = - Setting.intSetting("cluster.snapshot.info.max_concurrent_fetches", 5, 1, - Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final Setting INTERNAL_SNAPSHOT_INFO_MAX_CONCURRENT_FETCHES_SETTING = Setting.intSetting( + "cluster.snapshot.info.max_concurrent_fetches", + 5, + 1, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); private static final Logger logger = LogManager.getLogger(InternalSnapshotsInfoService.class); @@ -89,8 +93,8 @@ public InternalSnapshotsInfoService( this.repositoriesService = repositoriesServiceSupplier; this.rerouteService = rerouteServiceSupplier; this.knownSnapshotShards = ImmutableOpenMap.of(); - this.unknownSnapshotShards = new LinkedHashSet<>(); - this.failedSnapshotShards = new LinkedHashSet<>(); + this.unknownSnapshotShards = new LinkedHashSet<>(); + this.failedSnapshotShards = new LinkedHashSet<>(); this.queue = new LinkedList<>(); this.mutex = new Object(); this.activeFetches = 0; @@ -108,7 +112,7 @@ private void setMaxConcurrentFetches(Integer maxConcurrentFetches) { @Override public SnapshotShardSizeInfo snapshotShardSizes() { - synchronized (mutex){ + synchronized (mutex) { final ImmutableOpenMap.Builder snapshotShardSizes = ImmutableOpenMap.builder(knownSnapshotShards); if (failedSnapshotShards.isEmpty() == false) { for (SnapshotShard snapshotShard : failedSnapshotShards) { @@ -217,8 +221,9 @@ protected void doRun() throws Exception { removed = unknownSnapshotShards.remove(snapshotShard); assert removed : "snapshot shard to remove does not exist " + snapshotShardSize; if (isMaster) { - final ImmutableOpenMap.Builder newSnapshotShardSizes = - ImmutableOpenMap.builder(knownSnapshotShards); + final ImmutableOpenMap.Builder newSnapshotShardSizes = ImmutableOpenMap.builder( + knownSnapshotShards + ); updated = newSnapshotShardSizes.put(snapshotShard, snapshotShardSize) == null; assert updated : "snapshot shard size already exists for " + snapshotShard; knownSnapshotShards = newSnapshotShardSizes.build(); @@ -316,10 +321,13 @@ private static Set listOfSnapshotShards(final ClusterState state) final Set snapshotShards = new HashSet<>(); for (ShardRouting shardRouting : state.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED)) { if (shardRouting.primary() && shardRouting.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { - final RecoverySource.SnapshotRecoverySource snapshotRecoverySource = - (RecoverySource.SnapshotRecoverySource) shardRouting.recoverySource(); - final SnapshotShard snapshotShard = new SnapshotShard(snapshotRecoverySource.snapshot(), - snapshotRecoverySource.index(), shardRouting.shardId()); + final RecoverySource.SnapshotRecoverySource snapshotRecoverySource = (RecoverySource.SnapshotRecoverySource) shardRouting + .recoverySource(); + final SnapshotShard snapshotShard = new SnapshotShard( + snapshotRecoverySource.snapshot(), + snapshotRecoverySource.index(), + shardRouting.shardId() + ); snapshotShards.add(snapshotShard); } } @@ -359,9 +367,7 @@ public boolean equals(Object o) { return false; } final SnapshotShard that = (SnapshotShard) o; - return shardId.equals(that.shardId) - && snapshot.equals(that.snapshot) - && index.equals(that.index); + return shardId.equals(that.shardId) && snapshot.equals(that.snapshot) && index.equals(that.index); } @Override @@ -371,11 +377,7 @@ public int hashCode() { @Override public String toString() { - return "[" + - "snapshot=" + snapshot + - ", index=" + index + - ", shard=" + shardId + - ']'; + return "[" + "snapshot=" + snapshot + ", index=" + index + ", shard=" + shardId + ']'; } } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/InvalidSnapshotNameException.java b/server/src/main/java/org/elasticsearch/snapshots/InvalidSnapshotNameException.java index 9f93a79c30fdd..0ec165a975cb6 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/InvalidSnapshotNameException.java +++ b/server/src/main/java/org/elasticsearch/snapshots/InvalidSnapshotNameException.java @@ -32,4 +32,3 @@ public RestStatus status() { } } - diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java index e4eaf20b1be1b..c9ceaccb47fad 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java @@ -86,7 +86,7 @@ public int totalShards() { * @return number of failed shards */ public int failedShards() { - return totalShards - successfulShards; + return totalShards - successfulShards; } /** @@ -125,8 +125,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - private static final ObjectParser PARSER = new ObjectParser<>(RestoreInfo.class.getName(), - true, RestoreInfo::new); + private static final ObjectParser PARSER = new ObjectParser<>(RestoreInfo.class.getName(), true, RestoreInfo::new); static { ObjectParser shardsParser = new ObjectParser<>("shards", true, null); @@ -166,10 +165,10 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; RestoreInfo that = (RestoreInfo) o; - return totalShards == that.totalShards && - successfulShards == that.successfulShards && - Objects.equals(name, that.name) && - Objects.equals(indices, that.indices); + return totalShards == that.totalShards + && successfulShards == that.successfulShards + && Objects.equals(name, that.name) + && Objects.equals(indices, that.indices); } @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 54a5540161eb0..e6fb3b20aa747 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -129,17 +129,15 @@ public class RestoreService implements ClusterStateApplier { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(RestoreService.class); public static final Setting REFRESH_REPO_UUID_ON_RESTORE_SETTING = Setting.boolSetting( - "snapshot.refresh_repo_uuid_on_restore", - true, - Setting.Property.NodeScope, - Setting.Property.Dynamic); - - private static final Set UNMODIFIABLE_SETTINGS = unmodifiableSet(newHashSet( - SETTING_NUMBER_OF_SHARDS, - SETTING_VERSION_CREATED, - SETTING_INDEX_UUID, - SETTING_CREATION_DATE, - SETTING_HISTORY_UUID)); + "snapshot.refresh_repo_uuid_on_restore", + true, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + private static final Set UNMODIFIABLE_SETTINGS = unmodifiableSet( + newHashSet(SETTING_NUMBER_OF_SHARDS, SETTING_VERSION_CREATED, SETTING_INDEX_UUID, SETTING_CREATION_DATE, SETTING_HISTORY_UUID) + ); // It's OK to change some settings, but we shouldn't allow simply removing them private static final Set UNREMOVABLE_SETTINGS; @@ -195,9 +193,8 @@ public RestoreService( this.shardLimitValidator = shardLimitValidator; this.systemIndices = systemIndices; this.refreshRepositoryUuidOnRestore = REFRESH_REPO_UUID_ON_RESTORE_SETTING.get(clusterService.getSettings()); - clusterService.getClusterSettings().addSettingsUpdateConsumer( - REFRESH_REPO_UUID_ON_RESTORE_SETTING, - this::setRefreshRepositoryUuidOnRestore); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(REFRESH_REPO_UUID_ON_RESTORE_SETTING, this::setRefreshRepositoryUuidOnRestore); } /** @@ -218,9 +215,11 @@ public void restoreSnapshot(final RestoreSnapshotRequest request, final ActionLi * @param updater handler that allows callers to make modifications to {@link Metadata} * in the same cluster state update as the restore operation */ - public void restoreSnapshot(final RestoreSnapshotRequest request, - final ActionListener listener, - final BiConsumer updater) { + public void restoreSnapshot( + final RestoreSnapshotRequest request, + final ActionListener listener, + final BiConsumer updater + ) { try { // Try and fill in any missing repository UUIDs in case they're needed during the restore @@ -234,32 +233,36 @@ public void restoreSnapshot(final RestoreSnapshotRequest request, repository.getRepositoryData(repositoryDataListener); repositoryDataListener.whenComplete(repositoryData -> repositoryUuidRefreshListener.whenComplete(ignored -> { - final String snapshotName = request.snapshot(); - final Optional matchingSnapshotId = repositoryData.getSnapshotIds().stream() - .filter(s -> snapshotName.equals(s.getName())).findFirst(); - if (matchingSnapshotId.isPresent() == false) { - throw new SnapshotRestoreException(repositoryName, snapshotName, "snapshot does not exist"); - } + final String snapshotName = request.snapshot(); + final Optional matchingSnapshotId = repositoryData.getSnapshotIds() + .stream() + .filter(s -> snapshotName.equals(s.getName())) + .findFirst(); + if (matchingSnapshotId.isPresent() == false) { + throw new SnapshotRestoreException(repositoryName, snapshotName, "snapshot does not exist"); + } - final SnapshotId snapshotId = matchingSnapshotId.get(); - if (request.snapshotUuid() != null && request.snapshotUuid().equals(snapshotId.getUUID()) == false) { - throw new SnapshotRestoreException(repositoryName, snapshotName, - "snapshot UUID mismatch: expected [" + request.snapshotUuid() + "] but got [" - + snapshotId.getUUID() + "]"); - } - repository.getSnapshotInfo( - snapshotId, - ActionListener.wrap( - snapshotInfo -> startRestore(snapshotInfo, repository, request, repositoryData, updater, listener), - listener::onFailure - ) + final SnapshotId snapshotId = matchingSnapshotId.get(); + if (request.snapshotUuid() != null && request.snapshotUuid().equals(snapshotId.getUUID()) == false) { + throw new SnapshotRestoreException( + repositoryName, + snapshotName, + "snapshot UUID mismatch: expected [" + request.snapshotUuid() + "] but got [" + snapshotId.getUUID() + "]" ); - }, listener::onFailure), - listener::onFailure - ); + } + repository.getSnapshotInfo( + snapshotId, + ActionListener.wrap( + snapshotInfo -> startRestore(snapshotInfo, repository, request, repositoryData, updater, listener), + listener::onFailure + ) + ); + }, listener::onFailure), listener::onFailure); } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("[{}] failed to restore snapshot", - request.repository() + ":" + request.snapshot()), e); + logger.warn( + () -> new ParameterizedMessage("[{}] failed to restore snapshot", request.repository() + ":" + request.snapshot()), + e + ); listener.onFailure(e); } } @@ -278,12 +281,14 @@ public void restoreSnapshot(final RestoreSnapshotRequest request, * @param listener listener to resolve once restore has been started * @throws IOException on failure to load metadata from the repository */ - private void startRestore(SnapshotInfo snapshotInfo, - Repository repository, - RestoreSnapshotRequest request, - RepositoryData repositoryData, - BiConsumer updater, - ActionListener listener) throws IOException { + private void startRestore( + SnapshotInfo snapshotInfo, + Repository repository, + RestoreSnapshotRequest request, + RepositoryData repositoryData, + BiConsumer updater, + ActionListener listener + ) throws IOException { final SnapshotId snapshotId = snapshotInfo.snapshotId(); final String repositoryName = repository.getMetadata().name(); final Snapshot snapshot = new Snapshot(repositoryName, snapshotId); @@ -304,8 +309,14 @@ private void startRestore(SnapshotInfo snapshotInfo, List requestIndices = new ArrayList<>(Arrays.asList(request.indices())); // Get data stream metadata for requested data streams - Tuple, Map> result = - getDataStreamsToRestore(repository, snapshotId, snapshotInfo, globalMetadata, requestIndices, request.includeAliases()); + Tuple, Map> result = getDataStreamsToRestore( + repository, + snapshotId, + snapshotInfo, + globalMetadata, + requestIndices, + request.includeAliases() + ); Map dataStreamsToRestore = result.v1(); Map dataStreamAliasesToRestore = result.v2(); @@ -313,26 +324,31 @@ private void startRestore(SnapshotInfo snapshotInfo, requestIndices.removeAll(dataStreamsToRestore.keySet()); // And add the backing indices - Set dataStreamIndices = dataStreamsToRestore.values().stream() - .flatMap(ds -> ds.getIndices().stream()) - .map(Index::getName) - .collect(Collectors.toSet()); + Set dataStreamIndices = dataStreamsToRestore.values() + .stream() + .flatMap(ds -> ds.getIndices().stream()) + .map(Index::getName) + .collect(Collectors.toSet()); requestIndices.addAll(dataStreamIndices); // Determine system indices to restore from requested feature states final Map> featureStatesToRestore = getFeatureStatesToRestore(request, snapshotInfo, snapshot); - final Set featureStateIndices = featureStatesToRestore.values().stream() - .flatMap(Collection::stream) - .collect(Collectors.toSet()); + final Set featureStateIndices = featureStatesToRestore.values() + .stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); // Resolve the indices that were directly requested - final List requestedIndicesInSnapshot = filterIndices(snapshotInfo.indices(), requestIndices.toArray(String[]::new), - request.indicesOptions()); + final List requestedIndicesInSnapshot = filterIndices( + snapshotInfo.indices(), + requestIndices.toArray(String[]::new), + request.indicesOptions() + ); // Combine into the final list of indices to be restored final List requestedIndicesIncludingSystem = Stream.concat( - requestedIndicesInSnapshot.stream(), - featureStateIndices.stream() + requestedIndicesInSnapshot.stream(), + featureStateIndices.stream() ).distinct().collect(Collectors.toList()); final Set explicitlyRequestedSystemIndices = new HashSet<>(); @@ -349,28 +365,31 @@ private void startRestore(SnapshotInfo snapshotInfo, // log a deprecation warning if the any of the indexes to delete were included in the request and the snapshot // is from a version that should have feature states if (snapshotInfo.version().onOrAfter(Version.V_7_12_0) && explicitlyRequestedSystemIndices.isEmpty() == false) { - deprecationLogger.deprecate(DeprecationCategory.API, "restore-system-index-from-snapshot", - "Restoring system indices by name is deprecated. Use feature states instead. System indices: " - + explicitlyRequestedSystemIndices); + deprecationLogger.deprecate( + DeprecationCategory.API, + "restore-system-index-from-snapshot", + "Restoring system indices by name is deprecated. Use feature states instead. System indices: " + + explicitlyRequestedSystemIndices + ); } // Now we can start the actual restore process by adding shards to be recovered in the cluster state // and updating cluster metadata (global and index) as needed clusterService.submitStateUpdateTask( - "restore_snapshot[" + snapshotId.getName() + ']', - new RestoreSnapshotStateTask( - request, - snapshot, - featureStatesToRestore.keySet(), - // Apply renaming on index names, returning a map of names where - // the key is the renamed index and the value is the original name - renamedIndices(request, requestedIndicesIncludingSystem, dataStreamIndices, featureStateIndices, repositoryData), - snapshotInfo, - metadataBuilder.dataStreams(dataStreamsToRestore, dataStreamAliasesToRestore).build(), - dataStreamsToRestore.values(), - updater, - listener - ) + "restore_snapshot[" + snapshotId.getName() + ']', + new RestoreSnapshotStateTask( + request, + snapshot, + featureStatesToRestore.keySet(), + // Apply renaming on index names, returning a map of names where + // the key is the renamed index and the value is the original name + renamedIndices(request, requestedIndicesIncludingSystem, dataStreamIndices, featureStateIndices, repositoryData), + snapshotInfo, + metadataBuilder.dataStreams(dataStreamsToRestore, dataStreamAliasesToRestore).build(), + dataStreamsToRestore.values(), + updater, + listener + ) ); } @@ -398,17 +417,24 @@ static void refreshRepositoryUuids(boolean enabled, RepositoriesService reposito // We only care about BlobStoreRepositories because they're the only ones that can contain a searchable snapshot, and we only care // about ones with missing UUIDs. It's possible to have the UUID change from under us if, e.g., the repository was wiped by an // external force, but in this case any searchable snapshots are lost anyway so it doesn't really matter. - final List repositories = repositoriesService.getRepositories().values().stream() - .filter(repository -> repository instanceof BlobStoreRepository - && repository.getMetadata().uuid().equals(RepositoryData.MISSING_UUID)).collect(Collectors.toList()); + final List repositories = repositoriesService.getRepositories() + .values() + .stream() + .filter( + repository -> repository instanceof BlobStoreRepository + && repository.getMetadata().uuid().equals(RepositoryData.MISSING_UUID) + ) + .collect(Collectors.toList()); if (repositories.isEmpty()) { logger.debug("repository UUID refresh is not required"); refreshListener.onResponse(null); return; } - logger.info("refreshing repository UUIDs for repositories [{}]", - repositories.stream().map(repository -> repository.getMetadata().name()).collect(Collectors.joining(","))); + logger.info( + "refreshing repository UUIDs for repositories [{}]", + repositories.stream().map(repository -> repository.getMetadata().name()).collect(Collectors.joining(",")) + ); final ActionListener groupListener = new GroupedActionListener<>(new ActionListener>() { @Override public void onResponse(Collection ignored) { @@ -433,16 +459,21 @@ private boolean isSystemIndex(IndexMetadata indexMetadata) { return indexMetadata.isSystem() || systemIndices.isSystemName(indexMetadata.getIndex().getName()); } - private Tuple, Map> getDataStreamsToRestore(Repository repository, - SnapshotId snapshotId, - SnapshotInfo snapshotInfo, - Metadata globalMetadata, - List requestIndices, - boolean includeAliases) { + private Tuple, Map> getDataStreamsToRestore( + Repository repository, + SnapshotId snapshotId, + SnapshotInfo snapshotInfo, + Metadata globalMetadata, + List requestIndices, + boolean includeAliases + ) { Map dataStreams; Map dataStreamAliases; - List requestedDataStreams = filterIndices(snapshotInfo.dataStreams(), requestIndices.toArray(String[]::new), - IndicesOptions.fromOptions(true, true, true, true)); + List requestedDataStreams = filterIndices( + snapshotInfo.dataStreams(), + requestIndices.toArray(String[]::new), + IndicesOptions.fromOptions(true, true, true, true) + ); if (requestedDataStreams.isEmpty()) { dataStreams = Map.of(); dataStreamAliases = Map.of(); @@ -473,12 +504,16 @@ private Tuple, Map> getDataStre return new Tuple<>(dataStreams, dataStreamAliases); } - private Map> getFeatureStatesToRestore(RestoreSnapshotRequest request, SnapshotInfo snapshotInfo, - Snapshot snapshot) { + private Map> getFeatureStatesToRestore( + RestoreSnapshotRequest request, + SnapshotInfo snapshotInfo, + Snapshot snapshot + ) { if (snapshotInfo.featureStates() == null) { return Collections.emptyMap(); } - final Map> snapshotFeatureStates = snapshotInfo.featureStates().stream() + final Map> snapshotFeatureStates = snapshotInfo.featureStates() + .stream() .collect(Collectors.toMap(SnapshotFeatureInfo::getPluginName, SnapshotFeatureInfo::getIndices)); final Map> featureStatesToRestore; @@ -498,25 +533,38 @@ private Map> getFeatureStatesToRestore(RestoreSnapshotReque // Otherwise, handle the list of requested feature states final Set requestedStates = Set.of(requestedFeatureStates); if (requestedStates.contains(NO_FEATURE_STATES_VALUE)) { - throw new SnapshotRestoreException(snapshot, "the feature_states value [" + NO_FEATURE_STATES_VALUE + - "] indicates that no feature states should be restored, but other feature states were requested: " + requestedStates); + throw new SnapshotRestoreException( + snapshot, + "the feature_states value [" + + NO_FEATURE_STATES_VALUE + + "] indicates that no feature states should be restored, but other feature states were requested: " + + requestedStates + ); } if (snapshotFeatureStates.keySet().containsAll(requestedStates) == false) { Set nonExistingRequestedStates = new HashSet<>(requestedStates); nonExistingRequestedStates.removeAll(snapshotFeatureStates.keySet()); - throw new SnapshotRestoreException(snapshot, "requested feature states [" + nonExistingRequestedStates + - "] are not present in snapshot"); + throw new SnapshotRestoreException( + snapshot, + "requested feature states [" + nonExistingRequestedStates + "] are not present in snapshot" + ); } featureStatesToRestore = new HashMap<>(snapshotFeatureStates); featureStatesToRestore.keySet().retainAll(requestedStates); } - final List featuresNotOnThisNode = featureStatesToRestore.keySet().stream() + final List featuresNotOnThisNode = featureStatesToRestore.keySet() + .stream() .filter(featureName -> systemIndices.getFeatures().containsKey(featureName) == false) .collect(Collectors.toList()); if (featuresNotOnThisNode.isEmpty() == false) { - throw new SnapshotRestoreException(snapshot, "requested feature states " + featuresNotOnThisNode + " are present in " + - "snapshot but those features are not installed on the current master node"); + throw new SnapshotRestoreException( + snapshot, + "requested feature states " + + featuresNotOnThisNode + + " are present in " + + "snapshot but those features are not installed on the current master node" + ); } return featureStatesToRestore; } @@ -546,17 +594,25 @@ private Set resolveSystemIndicesToDelete(ClusterState currentState, Set updatedIndices = dataStream.getIndices().stream() + List updatedIndices = dataStream.getIndices() + .stream() .map(i -> metadata.get(renameIndex(i.getName(), request, true)).getIndex()) .collect(Collectors.toList()); - return new DataStream(dataStreamName, dataStream.getTimeStampField(), updatedIndices, dataStream.getGeneration(), - dataStream.getMetadata(), dataStream.isHidden(), dataStream.isReplicated()); + return new DataStream( + dataStreamName, + dataStream.getTimeStampField(), + updatedIndices, + dataStream.getGeneration(), + dataStream.getMetadata(), + dataStream.isHidden(), + dataStream.isReplicated() + ); } public static RestoreInProgress updateRestoreStateWithDeletedIndices(RestoreInProgress oldRestore, Set deletedIndices) { @@ -571,14 +627,20 @@ public static RestoreInProgress updateRestoreStateWithDeletedIndices(RestoreInPr if (shardsBuilder == null) { shardsBuilder = ImmutableOpenMap.builder(entry.shards()); } - shardsBuilder.put(shardId, - new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "index was deleted")); + shardsBuilder.put(shardId, new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "index was deleted")); } } if (shardsBuilder != null) { ImmutableOpenMap shards = shardsBuilder.build(); - builder.add(new RestoreInProgress.Entry(entry.uuid(), entry.snapshot(), - overallState(RestoreInProgress.State.STARTED, shards), entry.indices(), shards)); + builder.add( + new RestoreInProgress.Entry( + entry.uuid(), + entry.snapshot(), + overallState(RestoreInProgress.State.STARTED, shards), + entry.indices(), + shards + ) + ); } else { builder.add(entry); } @@ -626,7 +688,8 @@ public void shardStarted(ShardRouting initializingShard, ShardRouting startedSha if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { changes(recoverySource).put( initializingShard.shardId(), - new ShardRestoreStatus(initializingShard.currentNodeId(), RestoreInProgress.State.SUCCESS)); + new ShardRestoreStatus(initializingShard.currentNodeId(), RestoreInProgress.State.SUCCESS) + ); } } } @@ -641,8 +704,13 @@ public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) // however, we only want to acknowledge the restore operation once it has been successfully restored on another node. if (unassignedInfo.getFailure() != null && Lucene.isCorruptionException(unassignedInfo.getFailure().getCause())) { changes(recoverySource).put( - failedShard.shardId(), new ShardRestoreStatus(failedShard.currentNodeId(), - RestoreInProgress.State.FAILURE, unassignedInfo.getFailure().getCause().getMessage())); + failedShard.shardId(), + new ShardRestoreStatus( + failedShard.currentNodeId(), + RestoreInProgress.State.FAILURE, + unassignedInfo.getFailure().getCause().getMessage() + ) + ); } } } @@ -651,12 +719,15 @@ public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) @Override public void shardInitialized(ShardRouting unassignedShard, ShardRouting initializedShard) { // if we force an empty primary, we should also fail the restore entry - if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT && - initializedShard.recoverySource().getType() != RecoverySource.Type.SNAPSHOT) { + if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT + && initializedShard.recoverySource().getType() != RecoverySource.Type.SNAPSHOT) { changes(unassignedShard.recoverySource()).put( unassignedShard.shardId(), - new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, - "recovery source type changed from snapshot to " + initializedShard.recoverySource()) + new ShardRestoreStatus( + null, + RestoreInProgress.State.FAILURE, + "recovery source type changed from snapshot to " + initializedShard.recoverySource() + ) ); } } @@ -669,7 +740,8 @@ public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo n String reason = "shard could not be allocated to any of the nodes"; changes(recoverySource).put( unassignedShard.shardId(), - new ShardRestoreStatus(unassignedShard.currentNodeId(), RestoreInProgress.State.FAILURE, reason)); + new ShardRestoreStatus(unassignedShard.currentNodeId(), RestoreInProgress.State.FAILURE, reason) + ); } } } @@ -713,8 +785,10 @@ public RestoreInProgress applyChanges(final RestoreInProgress oldRestore) { } } - private static RestoreInProgress.State overallState(RestoreInProgress.State nonCompletedState, - ImmutableOpenMap shards) { + private static RestoreInProgress.State overallState( + RestoreInProgress.State nonCompletedState, + ImmutableOpenMap shards + ) { boolean hasFailed = false; for (ObjectCursor status : shards.values()) { if (status.value.state().completed() == false) { @@ -750,11 +824,13 @@ public static int failedShards(ImmutableOpenMap renamedIndices(RestoreSnapshotRequest request, - List filteredIndices, - Set dataStreamIndices, - Set featureIndices, - RepositoryData repositoryData) { + private static Map renamedIndices( + RestoreSnapshotRequest request, + List filteredIndices, + Set dataStreamIndices, + Set featureIndices, + RepositoryData repositoryData + ) { Map renamedIndices = new HashMap<>(); for (String index : filteredIndices) { String renamedIndex; @@ -767,7 +843,8 @@ private static Map renamedIndices(RestoreSnapshotRequest reques IndexId previousIndex = renamedIndices.put(renamedIndex, repositoryData.resolveIndexId(index)); if (previousIndex != null) { throw new SnapshotRestoreException( - request.repository(), request.snapshot(), + request.repository(), + request.snapshot(), "indices [" + index + "] and [" + previousIndex.getName() + "] are renamed into the same index [" + renamedIndex + "]" ); } @@ -798,19 +875,30 @@ private static String renameIndex(String index, RestoreSnapshotRequest request, */ private static void validateSnapshotRestorable(final String repository, final SnapshotInfo snapshotInfo) { if (snapshotInfo.state().restorable() == false) { - throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()), - "unsupported snapshot state [" + snapshotInfo.state() + "]"); + throw new SnapshotRestoreException( + new Snapshot(repository, snapshotInfo.snapshotId()), + "unsupported snapshot state [" + snapshotInfo.state() + "]" + ); } if (Version.CURRENT.before(snapshotInfo.version())) { - throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()), - "the snapshot was created with Elasticsearch version [" + snapshotInfo.version() + - "] which is higher than the version of this node [" + Version.CURRENT + "]"); + throw new SnapshotRestoreException( + new Snapshot(repository, snapshotInfo.snapshotId()), + "the snapshot was created with Elasticsearch version [" + + snapshotInfo.version() + + "] which is higher than the version of this node [" + + Version.CURRENT + + "]" + ); } if (snapshotInfo.version().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { - throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()), - "the snapshot was created with Elasticsearch version [" + snapshotInfo.version() + - "] which is below the current versions minimum index compatibility version [" + - Version.CURRENT.minimumIndexCompatibilityVersion() + "]"); + throw new SnapshotRestoreException( + new Snapshot(repository, snapshotInfo.snapshotId()), + "the snapshot was created with Elasticsearch version [" + + snapshotInfo.version() + + "] which is below the current versions minimum index compatibility version [" + + Version.CURRENT.minimumIndexCompatibilityVersion() + + "]" + ); } } @@ -865,8 +953,9 @@ public ClusterState execute(ClusterState currentState) { restoreInProgressBuilder.add(entry); } } - return changed == false ? currentState : ClusterState.builder(currentState).putCustom( - RestoreInProgress.TYPE, restoreInProgressBuilder.build()).build(); + return changed == false + ? currentState + : ClusterState.builder(currentState).putCustom(RestoreInProgress.TYPE, restoreInProgressBuilder.build()).build(); } @Override @@ -912,19 +1001,23 @@ public void applyClusterState(ClusterChangedEvent event) { * Optionally updates index settings in indexMetadata by removing settings listed in ignoreSettings and * merging them with settings in changeSettings. */ - private static IndexMetadata updateIndexSettings(Snapshot snapshot, - IndexMetadata indexMetadata, - Settings changeSettings, - String[] ignoreSettings) { + private static IndexMetadata updateIndexSettings( + Snapshot snapshot, + IndexMetadata indexMetadata, + Settings changeSettings, + String[] ignoreSettings + ) { Settings normalizedChangeSettings = Settings.builder() - .put(changeSettings) - .normalizePrefix(IndexMetadata.INDEX_SETTING_PREFIX) - .build(); - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(indexMetadata.getSettings()) && - IndexSettings.INDEX_SOFT_DELETES_SETTING.exists(changeSettings) && - IndexSettings.INDEX_SOFT_DELETES_SETTING.get(changeSettings) == false) { - throw new SnapshotRestoreException(snapshot, - "cannot disable setting [" + IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey() + "] on restore"); + .put(changeSettings) + .normalizePrefix(IndexMetadata.INDEX_SETTING_PREFIX) + .build(); + if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(indexMetadata.getSettings()) + && IndexSettings.INDEX_SOFT_DELETES_SETTING.exists(changeSettings) + && IndexSettings.INDEX_SOFT_DELETES_SETTING.get(changeSettings) == false) { + throw new SnapshotRestoreException( + snapshot, + "cannot disable setting [" + IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey() + "] on restore" + ); } IndexMetadata.Builder builder = IndexMetadata.builder(indexMetadata); Settings settings = indexMetadata.getSettings(); @@ -933,8 +1026,7 @@ private static IndexMetadata updateIndexSettings(Snapshot snapshot, for (String ignoredSetting : ignoreSettings) { if (Regex.isSimpleMatchPattern(ignoredSetting) == false) { if (UNREMOVABLE_SETTINGS.contains(ignoredSetting)) { - throw new SnapshotRestoreException( - snapshot, "cannot remove setting [" + ignoredSetting + "] on restore"); + throw new SnapshotRestoreException(snapshot, "cannot remove setting [" + ignoredSetting + "] on restore"); } else { keyFilters.add(ignoredSetting); } @@ -942,29 +1034,27 @@ private static IndexMetadata updateIndexSettings(Snapshot snapshot, simpleMatchPatterns.add(ignoredSetting); } } - Settings.Builder settingsBuilder = Settings.builder() - .put(settings.filter(k -> { - if (UNREMOVABLE_SETTINGS.contains(k) == false) { - for (String filterKey : keyFilters) { - if (k.equals(filterKey)) { - return false; - } - } - for (String pattern : simpleMatchPatterns) { - if (Regex.simpleMatch(pattern, k)) { - return false; - } - } + Settings.Builder settingsBuilder = Settings.builder().put(settings.filter(k -> { + if (UNREMOVABLE_SETTINGS.contains(k) == false) { + for (String filterKey : keyFilters) { + if (k.equals(filterKey)) { + return false; } - return true; - })) - .put(normalizedChangeSettings.filter(k -> { - if (UNMODIFIABLE_SETTINGS.contains(k)) { - throw new SnapshotRestoreException(snapshot, "cannot modify setting [" + k + "] on restore"); - } else { - return true; + } + for (String pattern : simpleMatchPatterns) { + if (Regex.simpleMatch(pattern, k)) { + return false; } - })); + } + } + return true; + })).put(normalizedChangeSettings.filter(k -> { + if (UNMODIFIABLE_SETTINGS.contains(k)) { + throw new SnapshotRestoreException(snapshot, "cannot modify setting [" + k + "] on restore"); + } else { + return true; + } + })); settingsBuilder.remove(MetadataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()); return builder.settings(settingsBuilder).build(); } @@ -1015,15 +1105,17 @@ private final class RestoreSnapshotStateTask extends ClusterStateUpdateTask { @Nullable private RestoreInfo restoreInfo; - RestoreSnapshotStateTask(RestoreSnapshotRequest request, - Snapshot snapshot, - Set featureStatesToRestore, - Map indicesToRestore, - SnapshotInfo snapshotInfo, - Metadata metadata, - Collection dataStreamsToRestore, - BiConsumer updater, - ActionListener listener) { + RestoreSnapshotStateTask( + RestoreSnapshotRequest request, + Snapshot snapshot, + Set featureStatesToRestore, + Map indicesToRestore, + SnapshotInfo snapshotInfo, + Metadata metadata, + Collection dataStreamsToRestore, + BiConsumer updater, + ActionListener listener + ) { super(request.masterNodeTimeout()); this.request = request; this.snapshot = snapshot; @@ -1043,8 +1135,8 @@ public ClusterState execute(ClusterState currentState) { // Clear out all existing indices which fall within a system index pattern being restored currentState = metadataDeleteIndexService.deleteIndices( - currentState, - resolveSystemIndicesToDelete(currentState, featureStatesToRestore) + currentState, + resolveSystemIndicesToDelete(currentState, featureStatesToRestore) ); // Updating cluster state @@ -1059,24 +1151,23 @@ public ClusterState execute(ClusterState currentState) { for (Map.Entry indexEntry : indicesToRestore.entrySet()) { final IndexId index = indexEntry.getValue(); IndexMetadata snapshotIndexMetadata = updateIndexSettings( - snapshot, - metadata.index(index.getName()), - request.indexSettings(), - request.ignoreIndexSettings() + snapshot, + metadata.index(index.getName()), + request.indexSettings(), + request.ignoreIndexSettings() ); try { snapshotIndexMetadata = indexMetadataVerifier.verifyIndexMetadata(snapshotIndexMetadata, minIndexCompatibilityVersion); } catch (Exception ex) { - throw new SnapshotRestoreException(snapshot, "cannot restore index [" + index + - "] because it cannot be upgraded", ex); + throw new SnapshotRestoreException(snapshot, "cannot restore index [" + index + "] because it cannot be upgraded", ex); } final String renamedIndexName = indexEntry.getKey(); final IndexMetadata currentIndexMetadata = currentState.metadata().index(renamedIndexName); final SnapshotRecoverySource recoverySource = new SnapshotRecoverySource( - restoreUUID, - snapshot, - snapshotInfo.version(), - index + restoreUUID, + snapshot, + snapshotInfo.version(), + index ); final boolean partial = checkPartial(index.getName()); final IntSet ignoreShards = new IntHashSet(); @@ -1091,8 +1182,9 @@ public ClusterState execute(ClusterState currentState) { shardLimitValidator.validateShardLimit(snapshotIndexMetadata.getSettings(), currentState); final IndexMetadata.Builder indexMdBuilder = restoreToCreateNewIndex(snapshotIndexMetadata, renamedIndexName); - if (request.includeAliases() == false && snapshotIndexMetadata.getAliases().isEmpty() == false - && isSystemIndex(snapshotIndexMetadata) == false) { + if (request.includeAliases() == false + && snapshotIndexMetadata.getAliases().isEmpty() == false + && isSystemIndex(snapshotIndexMetadata) == false) { // Remove all aliases - they shouldn't be restored indexMdBuilder.removeAllAliases(); } else { @@ -1130,10 +1222,10 @@ && isSystemIndex(snapshotIndexMetadata) == false) { final Index renamedIndex = updatedIndexMetadata.getIndex(); for (int shard = 0; shard < snapshotIndexMetadata.getNumberOfShards(); shard++) { shardsBuilder.put( - new ShardId(renamedIndex, shard), - ignoreShards.contains(shard) - ? new ShardRestoreStatus(localNodeId, RestoreInProgress.State.FAILURE) - : new ShardRestoreStatus(localNodeId) + new ShardId(renamedIndex, shard), + ignoreShards.contains(shard) + ? new ShardRestoreStatus(localNodeId, RestoreInProgress.State.FAILURE) + : new ShardRestoreStatus(localNodeId) ); } } @@ -1142,16 +1234,16 @@ && isSystemIndex(snapshotIndexMetadata) == false) { final ImmutableOpenMap shards = shardsBuilder.build(); if (shards.isEmpty() == false) { builder.putCustom( - RestoreInProgress.TYPE, - new RestoreInProgress.Builder(currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)).add( - new RestoreInProgress.Entry( - restoreUUID, - snapshot, - overallState(RestoreInProgress.State.INIT, shards), - List.copyOf(indicesToRestore.keySet()), - shards - ) - ).build() + RestoreInProgress.TYPE, + new RestoreInProgress.Builder(currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)).add( + new RestoreInProgress.Entry( + restoreUUID, + snapshot, + overallState(RestoreInProgress.State.INIT, shards), + List.copyOf(indicesToRestore.keySet()), + shards + ) + ).build() ); } @@ -1164,26 +1256,32 @@ && isSystemIndex(snapshotIndexMetadata) == false) { if (completed(shards)) { // We don't have any indices to restore - we are done - restoreInfo = new RestoreInfo(snapshot.getSnapshotId().getName(), + restoreInfo = new RestoreInfo( + snapshot.getSnapshotId().getName(), List.copyOf(indicesToRestore.keySet()), shards.size(), - shards.size() - failedShards(shards)); + shards.size() - failedShards(shards) + ); } updater.accept(currentState, mdBuilder); return allocationService.reroute( - builder.metadata(mdBuilder).blocks(blocks).routingTable(rtBuilder.build()).build(), - "restored snapshot [" + snapshot + "]" + builder.metadata(mdBuilder).blocks(blocks).routingTable(rtBuilder.build()).build(), + "restored snapshot [" + snapshot + "]" ); } private void applyDataStreamRestores(ClusterState currentState, Metadata.Builder mdBuilder) { final Map updatedDataStreams = new HashMap<>(currentState.metadata().dataStreams()); - updatedDataStreams.putAll(dataStreamsToRestore.stream() - .map(ds -> updateDataStream(ds, mdBuilder, request)) - .collect(Collectors.toMap(DataStream::getName, Function.identity()))); + updatedDataStreams.putAll( + dataStreamsToRestore.stream() + .map(ds -> updateDataStream(ds, mdBuilder, request)) + .collect(Collectors.toMap(DataStream::getName, Function.identity())) + ); final Map updatedDataStreamAliases = new HashMap<>(currentState.metadata().dataStreamAliases()); - metadata.dataStreamAliases().values().stream() + metadata.dataStreamAliases() + .values() + .stream() // Optionally rename the data stream names for each alias .map(alias -> { if (request.renamePattern() != null && request.renameReplacement() != null) { @@ -1191,7 +1289,8 @@ private void applyDataStreamRestores(ClusterState currentState, Metadata.Builder } else { return alias; } - }).forEach(alias -> { + }) + .forEach(alias -> { final DataStreamAlias current = updatedDataStreamAliases.putIfAbsent(alias.getName(), alias); if (current != null) { // Merge data stream alias from snapshot with an existing data stream aliases in target cluster: @@ -1203,12 +1302,15 @@ private void applyDataStreamRestores(ClusterState currentState, Metadata.Builder } private void ensureSnapshotNotDeleted(ClusterState currentState) { - SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(snapshot.getSnapshotId()))) { - throw new ConcurrentSnapshotExecutionException(snapshot, - "cannot restore a snapshot while a snapshot deletion is in-progress [" + - deletionsInProgress.getEntries().get(0) + "]"); + throw new ConcurrentSnapshotExecutionException( + snapshot, + "cannot restore a snapshot while a snapshot deletion is in-progress [" + deletionsInProgress.getEntries().get(0) + "]" + ); } } @@ -1218,12 +1320,12 @@ private void applyGlobalStateRestore(ClusterState currentState, Metadata.Builder if (request.skipOperatorOnlyState()) { // Skip any operator-only settings from the snapshot. This happens when operator privileges are enabled final Set operatorSettingKeys = Stream.concat( - settings.keySet().stream(), currentState.metadata().persistentSettings().keySet().stream()) - .filter(k -> { - final Setting setting = clusterSettings.get(k); - return setting != null && setting.isOperatorOnly(); - }) - .collect(Collectors.toSet()); + settings.keySet().stream(), + currentState.metadata().persistentSettings().keySet().stream() + ).filter(k -> { + final Setting setting = clusterSettings.get(k); + return setting != null && setting.isOperatorOnly(); + }).collect(Collectors.toSet()); if (false == operatorSettingKeys.isEmpty()) { settings = Settings.builder() .put(settings.filter(k -> false == operatorSettingKeys.contains(k))) @@ -1243,8 +1345,8 @@ private void applyGlobalStateRestore(ClusterState currentState, Metadata.Builder if (metadata.customs() != null) { for (ObjectObjectCursor cursor : metadata.customs()) { if (RepositoriesMetadata.TYPE.equals(cursor.key) == false - && DataStreamMetadata.TYPE.equals(cursor.key) == false - && cursor.value instanceof Metadata.NonRestorableCustom == false) { + && DataStreamMetadata.TYPE.equals(cursor.key) == false + && cursor.value instanceof Metadata.NonRestorableCustom == false) { // TODO: Check request.skipOperatorOnly for Autoscaling policies (NonRestorableCustom) // Don't restore repositories while we are working with them // TODO: Should we restore them at the end? @@ -1260,9 +1362,14 @@ private void ensureNoAliasNameConflicts(IndexMetadata snapshotIndexMetadata) { final String aliasName = alias.value; final IndexId indexId = indicesToRestore.get(aliasName); if (indexId != null) { - throw new SnapshotRestoreException(snapshot, - "cannot rename index [" + indexId + "] into [" + aliasName - + "] because of conflict with an alias with the same name"); + throw new SnapshotRestoreException( + snapshot, + "cannot rename index [" + + indexId + + "] into [" + + aliasName + + "] because of conflict with an alias with the same name" + ); } } } @@ -1288,27 +1395,45 @@ private boolean checkPartial(String index) { } } - private void validateExistingClosedIndex(IndexMetadata currentIndexMetadata, IndexMetadata snapshotIndexMetadata, - String renamedIndex, boolean partial) { + private void validateExistingClosedIndex( + IndexMetadata currentIndexMetadata, + IndexMetadata snapshotIndexMetadata, + String renamedIndex, + boolean partial + ) { // Index exist - checking that it's closed if (currentIndexMetadata.getState() != IndexMetadata.State.CLOSE) { // TODO: Enable restore for open indices - throw new SnapshotRestoreException(snapshot, "cannot restore index [" + renamedIndex - + "] because an open index " + - "with same name already exists in the cluster. Either close or delete the existing index or restore the " + - "index under a different name by providing a rename pattern and replacement name"); + throw new SnapshotRestoreException( + snapshot, + "cannot restore index [" + + renamedIndex + + "] because an open index " + + "with same name already exists in the cluster. Either close or delete the existing index or restore the " + + "index under a different name by providing a rename pattern and replacement name" + ); } // Index exist - checking if it's partial restore if (partial) { - throw new SnapshotRestoreException(snapshot, "cannot restore partial index [" + renamedIndex - + "] because such index already exists"); + throw new SnapshotRestoreException( + snapshot, + "cannot restore partial index [" + renamedIndex + "] because such index already exists" + ); } // Make sure that the number of shards is the same. That's the only thing that we cannot change if (currentIndexMetadata.getNumberOfShards() != snapshotIndexMetadata.getNumberOfShards()) { - throw new SnapshotRestoreException(snapshot, - "cannot restore index [" + renamedIndex + "] with [" + currentIndexMetadata.getNumberOfShards() - + "] shards from a snapshot of index [" + snapshotIndexMetadata.getIndex().getName() + "] with [" + - snapshotIndexMetadata.getNumberOfShards() + "] shards"); + throw new SnapshotRestoreException( + snapshot, + "cannot restore index [" + + renamedIndex + + "] with [" + + currentIndexMetadata.getNumberOfShards() + + "] shards from a snapshot of index [" + + snapshotIndexMetadata.getIndex().getName() + + "] with [" + + snapshotIndexMetadata.getNumberOfShards() + + "] shards" + ); } } @@ -1326,33 +1451,31 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS private static IndexMetadata.Builder restoreToCreateNewIndex(IndexMetadata snapshotIndexMetadata, String renamedIndexName) { return IndexMetadata.builder(snapshotIndexMetadata) - .state(IndexMetadata.State.OPEN) - .index(renamedIndexName) - .settings( - Settings.builder() - .put(snapshotIndexMetadata.getSettings()) - .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) - ).timestampRange(IndexLongFieldRange.NO_SHARDS); + .state(IndexMetadata.State.OPEN) + .index(renamedIndexName) + .settings( + Settings.builder().put(snapshotIndexMetadata.getSettings()).put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + ) + .timestampRange(IndexLongFieldRange.NO_SHARDS); } private static IndexMetadata.Builder restoreOverClosedIndex(IndexMetadata snapshotIndexMetadata, IndexMetadata currentIndexMetadata) { final IndexMetadata.Builder indexMdBuilder = IndexMetadata.builder(snapshotIndexMetadata) - .state(IndexMetadata.State.OPEN) - .version(Math.max(snapshotIndexMetadata.getVersion(), 1 + currentIndexMetadata.getVersion())) - .mappingVersion(Math.max(snapshotIndexMetadata.getMappingVersion(), 1 + currentIndexMetadata.getMappingVersion())) - .settingsVersion(Math.max(snapshotIndexMetadata.getSettingsVersion(), 1 + currentIndexMetadata.getSettingsVersion())) - .aliasesVersion(Math.max(snapshotIndexMetadata.getAliasesVersion(), 1 + currentIndexMetadata.getAliasesVersion())) - .timestampRange(IndexLongFieldRange.NO_SHARDS) - .index(currentIndexMetadata.getIndex().getName()) - .settings( - Settings.builder() - .put(snapshotIndexMetadata.getSettings()) - .put(IndexMetadata.SETTING_INDEX_UUID, currentIndexMetadata.getIndexUUID()) - .put(IndexMetadata.SETTING_HISTORY_UUID, UUIDs.randomBase64UUID()) - ); + .state(IndexMetadata.State.OPEN) + .version(Math.max(snapshotIndexMetadata.getVersion(), 1 + currentIndexMetadata.getVersion())) + .mappingVersion(Math.max(snapshotIndexMetadata.getMappingVersion(), 1 + currentIndexMetadata.getMappingVersion())) + .settingsVersion(Math.max(snapshotIndexMetadata.getSettingsVersion(), 1 + currentIndexMetadata.getSettingsVersion())) + .aliasesVersion(Math.max(snapshotIndexMetadata.getAliasesVersion(), 1 + currentIndexMetadata.getAliasesVersion())) + .timestampRange(IndexLongFieldRange.NO_SHARDS) + .index(currentIndexMetadata.getIndex().getName()) + .settings( + Settings.builder() + .put(snapshotIndexMetadata.getSettings()) + .put(IndexMetadata.SETTING_INDEX_UUID, currentIndexMetadata.getIndexUUID()) + .put(IndexMetadata.SETTING_HISTORY_UUID, UUIDs.randomBase64UUID()) + ); for (int shard = 0; shard < snapshotIndexMetadata.getNumberOfShards(); shard++) { - indexMdBuilder.primaryTerm(shard, - Math.max(snapshotIndexMetadata.primaryTerm(shard), currentIndexMetadata.primaryTerm(shard))); + indexMdBuilder.primaryTerm(shard, Math.max(snapshotIndexMetadata.primaryTerm(shard), currentIndexMetadata.primaryTerm(shard))); } return indexMdBuilder; } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotFeatureInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotFeatureInfo.java index 419d75a7226a5..95d93db7df04c 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotFeatureInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotFeatureInfo.java @@ -26,12 +26,15 @@ public class SnapshotFeatureInfo implements Writeable, ToXContentObject { final String pluginName; final List indices; - static final ConstructingObjectParser SNAPSHOT_FEATURE_INFO_PARSER = - new ConstructingObjectParser<>("feature_info", true, (a, name) -> { + static final ConstructingObjectParser SNAPSHOT_FEATURE_INFO_PARSER = new ConstructingObjectParser<>( + "feature_info", + true, + (a, name) -> { String pluginName = (String) a[0]; List indices = (List) a[1]; return new SnapshotFeatureInfo(pluginName, indices); - }); + } + ); static { SNAPSHOT_FEATURE_INFO_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("feature_name")); @@ -91,8 +94,7 @@ public boolean equals(Object o) { if (this == o) return true; if ((o instanceof SnapshotFeatureInfo) == false) return false; SnapshotFeatureInfo that = (SnapshotFeatureInfo) o; - return getPluginName().equals(that.getPluginName()) && - getIndices().equals(that.getIndices()); + return getPluginName().equals(that.getPluginName()) && getIndices().equals(that.getIndices()); } @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInProgressException.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInProgressException.java index 3f2d9a4182ca8..390178fade4f8 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInProgressException.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInProgressException.java @@ -33,4 +33,3 @@ public RestStatus status() { return RestStatus.BAD_REQUEST; } } - diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index 617ac8390178a..9dadf5eecaf4a 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -76,8 +76,8 @@ public final class SnapshotInfo implements Comparable, ToXContent, private static final String FEATURE_STATES = "feature_states"; private static final String INDEX_DETAILS = "index_details"; - private static final Comparator COMPARATOR = - Comparator.comparing(SnapshotInfo::startTime).thenComparing(SnapshotInfo::snapshotId); + private static final Comparator COMPARATOR = Comparator.comparing(SnapshotInfo::startTime) + .thenComparing(SnapshotInfo::snapshotId); public static final class SnapshotInfoBuilder { private String snapshotName = null; @@ -186,21 +186,21 @@ public SnapshotInfo build() { } return new SnapshotInfo( - snapshotId, - indices, - dataStreams, - featureStates, - reason, - version, - startTime, - endTime, - totalShards, - successfulShards, - shardFailures, - includeGlobalState, - userMetadata, - snapshotState, - indexSnapshotDetails + snapshotId, + indices, + dataStreams, + featureStates, + reason, + version, + startTime, + endTime, + totalShards, + successfulShards, + shardFailures, + includeGlobalState, + userMetadata, + snapshotState, + indexSnapshotDetails ); } } @@ -226,11 +226,17 @@ int getSuccessfulShards() { } } - public static final ObjectParser SNAPSHOT_INFO_PARSER = - new ObjectParser<>(SnapshotInfoBuilder.class.getName(), true, SnapshotInfoBuilder::new); + public static final ObjectParser SNAPSHOT_INFO_PARSER = new ObjectParser<>( + SnapshotInfoBuilder.class.getName(), + true, + SnapshotInfoBuilder::new + ); - private static final ObjectParser SHARD_STATS_PARSER = - new ObjectParser<>(ShardStatsBuilder.class.getName(), true, ShardStatsBuilder::new); + private static final ObjectParser SHARD_STATS_PARSER = new ObjectParser<>( + ShardStatsBuilder.class.getName(), + true, + ShardStatsBuilder::new + ); static { SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setSnapshotName, new ParseField(SNAPSHOT)); @@ -239,20 +245,27 @@ int getSuccessfulShards() { SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setReason, new ParseField(REASON)); SNAPSHOT_INFO_PARSER.declareStringArray(SnapshotInfoBuilder::setIndices, new ParseField(INDICES)); SNAPSHOT_INFO_PARSER.declareStringArray(SnapshotInfoBuilder::setDataStreams, new ParseField(DATA_STREAMS)); - SNAPSHOT_INFO_PARSER.declareObjectArray(SnapshotInfoBuilder::setFeatureStates, SnapshotFeatureInfo.SNAPSHOT_FEATURE_INFO_PARSER, - new ParseField(FEATURE_STATES)); + SNAPSHOT_INFO_PARSER.declareObjectArray( + SnapshotInfoBuilder::setFeatureStates, + SnapshotFeatureInfo.SNAPSHOT_FEATURE_INFO_PARSER, + new ParseField(FEATURE_STATES) + ); SNAPSHOT_INFO_PARSER.declareObject( - SnapshotInfoBuilder::setIndexSnapshotDetails, - (p, c) -> p.map(HashMap::new, p2 -> IndexSnapshotDetails.PARSER.parse(p2, c)), - new ParseField(INDEX_DETAILS)); + SnapshotInfoBuilder::setIndexSnapshotDetails, + (p, c) -> p.map(HashMap::new, p2 -> IndexSnapshotDetails.PARSER.parse(p2, c)), + new ParseField(INDEX_DETAILS) + ); SNAPSHOT_INFO_PARSER.declareLong(SnapshotInfoBuilder::setStartTime, new ParseField(START_TIME_IN_MILLIS)); SNAPSHOT_INFO_PARSER.declareLong(SnapshotInfoBuilder::setEndTime, new ParseField(END_TIME_IN_MILLIS)); SNAPSHOT_INFO_PARSER.declareObject(SnapshotInfoBuilder::setShardStatsBuilder, SHARD_STATS_PARSER, new ParseField(SHARDS)); SNAPSHOT_INFO_PARSER.declareBoolean(SnapshotInfoBuilder::setIncludeGlobalState, new ParseField(INCLUDE_GLOBAL_STATE)); - SNAPSHOT_INFO_PARSER.declareObject(SnapshotInfoBuilder::setUserMetadata, (p, c) -> p.map() , new ParseField(USER_METADATA)); + SNAPSHOT_INFO_PARSER.declareObject(SnapshotInfoBuilder::setUserMetadata, (p, c) -> p.map(), new ParseField(USER_METADATA)); SNAPSHOT_INFO_PARSER.declareInt(SnapshotInfoBuilder::setVersion, new ParseField(VERSION_ID)); - SNAPSHOT_INFO_PARSER.declareObjectArray(SnapshotInfoBuilder::setShardFailures, SnapshotShardFailure.SNAPSHOT_SHARD_FAILURE_PARSER, - new ParseField(FAILURES)); + SNAPSHOT_INFO_PARSER.declareObjectArray( + SnapshotInfoBuilder::setShardFailures, + SnapshotShardFailure.SNAPSHOT_SHARD_FAILURE_PARSER, + new ParseField(FAILURES) + ); SHARD_STATS_PARSER.declareInt(ShardStatsBuilder::setTotalShards, new ParseField(TOTAL)); SHARD_STATS_PARSER.declareInt(ShardStatsBuilder::setSuccessfulShards, new ParseField(SUCCESSFUL)); @@ -294,124 +307,128 @@ int getSuccessfulShards() { private final Map indexSnapshotDetails; public SnapshotInfo( - SnapshotId snapshotId, - List indices, - List dataStreams, - List featureStates, - SnapshotState state) { + SnapshotId snapshotId, + List indices, + List dataStreams, + List featureStates, + SnapshotState state + ) { this( - snapshotId, - indices, - dataStreams, - featureStates, - null, - null, - 0L, - 0L, - 0, - 0, - Collections.emptyList(), - null, - null, - state, - Collections.emptyMap() + snapshotId, + indices, + dataStreams, + featureStates, + null, + null, + 0L, + 0L, + 0, + 0, + Collections.emptyList(), + null, + null, + state, + Collections.emptyMap() ); } public SnapshotInfo( - SnapshotId snapshotId, - List indices, - List dataStreams, - List featureStates, - Version version, - SnapshotState state) { + SnapshotId snapshotId, + List indices, + List dataStreams, + List featureStates, + Version version, + SnapshotState state + ) { this( - snapshotId, - indices, - dataStreams, - featureStates, - null, - version, - 0L, - 0L, - 0, - 0, - Collections.emptyList(), - null, - null, - state, - Collections.emptyMap() + snapshotId, + indices, + dataStreams, + featureStates, + null, + version, + 0L, + 0L, + 0, + 0, + Collections.emptyList(), + null, + null, + state, + Collections.emptyMap() ); } public SnapshotInfo(SnapshotsInProgress.Entry entry) { this( - entry.snapshot().getSnapshotId(), - List.copyOf(entry.indices().keySet()), - entry.dataStreams(), - entry.featureStates(), - null, - Version.CURRENT, - entry.startTime(), - 0L, - 0, - 0, - Collections.emptyList(), - entry.includeGlobalState(), - entry.userMetadata(), - SnapshotState.IN_PROGRESS, - Collections.emptyMap() + entry.snapshot().getSnapshotId(), + List.copyOf(entry.indices().keySet()), + entry.dataStreams(), + entry.featureStates(), + null, + Version.CURRENT, + entry.startTime(), + 0L, + 0, + 0, + Collections.emptyList(), + entry.includeGlobalState(), + entry.userMetadata(), + SnapshotState.IN_PROGRESS, + Collections.emptyMap() ); } public SnapshotInfo( - SnapshotId snapshotId, - List indices, - List dataStreams, - List featureStates, - String reason, - long endTime, - int totalShards, - List shardFailures, - Boolean includeGlobalState, - Map userMetadata, - long startTime, - Map indexSnapshotDetails) { + SnapshotId snapshotId, + List indices, + List dataStreams, + List featureStates, + String reason, + long endTime, + int totalShards, + List shardFailures, + Boolean includeGlobalState, + Map userMetadata, + long startTime, + Map indexSnapshotDetails + ) { this( - snapshotId, - indices, - dataStreams, - featureStates, - reason, - Version.CURRENT, - startTime, - endTime, - totalShards, - totalShards - shardFailures.size(), - shardFailures, - includeGlobalState, - userMetadata, - snapshotState(reason, shardFailures), - indexSnapshotDetails + snapshotId, + indices, + dataStreams, + featureStates, + reason, + Version.CURRENT, + startTime, + endTime, + totalShards, + totalShards - shardFailures.size(), + shardFailures, + includeGlobalState, + userMetadata, + snapshotState(reason, shardFailures), + indexSnapshotDetails ); } SnapshotInfo( - SnapshotId snapshotId, - List indices, - List dataStreams, - List featureStates, - String reason, - Version version, - long startTime, - long endTime, - int totalShards, - int successfulShards, - List shardFailures, - Boolean includeGlobalState, - Map userMetadata, - SnapshotState state, - Map indexSnapshotDetails) { + SnapshotId snapshotId, + List indices, + List dataStreams, + List featureStates, + String reason, + Version version, + long startTime, + long endTime, + int totalShards, + int successfulShards, + List shardFailures, + Boolean includeGlobalState, + Map userMetadata, + SnapshotState state, + Map indexSnapshotDetails + ) { this.snapshotId = Objects.requireNonNull(snapshotId); this.indices = Collections.unmodifiableList(Objects.requireNonNull(indices)); this.dataStreams = Collections.unmodifiableList(Objects.requireNonNull(dataStreams)); @@ -499,7 +516,7 @@ public List indices() { /** * @return list of data streams that were included in this snapshot. */ - public List dataStreams(){ + public List dataStreams() { return dataStreams; } @@ -609,21 +626,35 @@ public int compareTo(final SnapshotInfo o) { @Override public String toString() { - return "SnapshotInfo{" + - "snapshotId=" + snapshotId + - ", state=" + state + - ", reason='" + reason + '\'' + - ", indices=" + indices + - ", startTime=" + startTime + - ", endTime=" + endTime + - ", totalShards=" + totalShards + - ", successfulShards=" + successfulShards + - ", includeGlobalState=" + includeGlobalState + - ", version=" + version + - ", shardFailures=" + shardFailures + - ", featureStates=" + featureStates + - ", indexSnapshotDetails=" + indexSnapshotDetails + - '}'; + return "SnapshotInfo{" + + "snapshotId=" + + snapshotId + + ", state=" + + state + + ", reason='" + + reason + + '\'' + + ", indices=" + + indices + + ", startTime=" + + startTime + + ", endTime=" + + endTime + + ", totalShards=" + + totalShards + + ", successfulShards=" + + successfulShards + + ", includeGlobalState=" + + includeGlobalState + + ", version=" + + version + + ", shardFailures=" + + shardFailures + + ", featureStates=" + + featureStates + + ", indexSnapshotDetails=" + + indexSnapshotDetails + + '}'; } /** @@ -636,8 +667,11 @@ public RestStatus status() { if (shardFailures.size() == 0) { return RestStatus.OK; } - return RestStatus.status(successfulShards, totalShards, - shardFailures.toArray(new ShardOperationFailedException[shardFailures.size()])); + return RestStatus.status( + successfulShards, + totalShards, + shardFailures.toArray(new ShardOperationFailedException[shardFailures.size()]) + ); } @Override @@ -876,21 +910,23 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr // the old format where there wasn't a UUID uuid = name; } - return new SnapshotInfo(new SnapshotId(name, uuid), - indices, - dataStreams, - featureStates, - reason, - version, - startTime, - endTime, - totalShards, - successfulShards, - shardFailures, - includeGlobalState, - userMetadata, - state, - indexSnapshotDetails == null ? Collections.emptyMap() : indexSnapshotDetails); + return new SnapshotInfo( + new SnapshotId(name, uuid), + indices, + dataStreams, + featureStates, + reason, + version, + startTime, + endTime, + totalShards, + successfulShards, + shardFailures, + includeGlobalState, + userMetadata, + state, + indexSnapshotDetails == null ? Collections.emptyMap() : indexSnapshotDetails + ); } @Override @@ -940,28 +976,42 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; SnapshotInfo that = (SnapshotInfo) o; - return startTime == that.startTime && - endTime == that.endTime && - totalShards == that.totalShards && - successfulShards == that.successfulShards && - Objects.equals(snapshotId, that.snapshotId) && - state == that.state && - Objects.equals(reason, that.reason) && - Objects.equals(indices, that.indices) && - Objects.equals(dataStreams, that.dataStreams) && - Objects.equals(includeGlobalState, that.includeGlobalState) && - Objects.equals(version, that.version) && - Objects.equals(shardFailures, that.shardFailures) && - Objects.equals(userMetadata, that.userMetadata) && - Objects.equals(featureStates, that.featureStates) && - Objects.equals(indexSnapshotDetails, that.indexSnapshotDetails); + return startTime == that.startTime + && endTime == that.endTime + && totalShards == that.totalShards + && successfulShards == that.successfulShards + && Objects.equals(snapshotId, that.snapshotId) + && state == that.state + && Objects.equals(reason, that.reason) + && Objects.equals(indices, that.indices) + && Objects.equals(dataStreams, that.dataStreams) + && Objects.equals(includeGlobalState, that.includeGlobalState) + && Objects.equals(version, that.version) + && Objects.equals(shardFailures, that.shardFailures) + && Objects.equals(userMetadata, that.userMetadata) + && Objects.equals(featureStates, that.featureStates) + && Objects.equals(indexSnapshotDetails, that.indexSnapshotDetails); } @Override public int hashCode() { - return Objects.hash(snapshotId, state, reason, indices, dataStreams, startTime, endTime, - totalShards, successfulShards, includeGlobalState, version, shardFailures, userMetadata, - featureStates, indexSnapshotDetails); + return Objects.hash( + snapshotId, + state, + reason, + indices, + dataStreams, + startTime, + endTime, + totalShards, + successfulShards, + includeGlobalState, + version, + shardFailures, + userMetadata, + featureStates, + indexSnapshotDetails + ); } public static class IndexSnapshotDetails implements ToXContentObject, Writeable { @@ -972,9 +1022,10 @@ public static class IndexSnapshotDetails implements ToXContentObject, Writeable public static final IndexSnapshotDetails SKIPPED = new IndexSnapshotDetails(0, ByteSizeValue.ZERO, 0); public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - IndexSnapshotDetails.class.getName(), - true, - a -> new IndexSnapshotDetails((int)a[0], ByteSizeValue.ofBytes((long) a[1]), (int)a[2])); + IndexSnapshotDetails.class.getName(), + true, + a -> new IndexSnapshotDetails((int) a[0], ByteSizeValue.ofBytes((long) a[1]), (int) a[2]) + ); static { PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(SHARD_COUNT)); @@ -1025,11 +1076,14 @@ public int hashCode() { @Override public String toString() { - return "IndexSnapshotDetails{" + - "shardCount=" + shardCount + - ", size=" + size + - ", maxSegmentsPerShard=" + maxSegmentsPerShard + - '}'; + return "IndexSnapshotDetails{" + + "shardCount=" + + shardCount + + ", size=" + + size + + ", maxSegmentsPerShard=" + + maxSegmentsPerShard + + '}'; } @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java index e14eee4352c6b..86e976b870301 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java @@ -92,16 +92,25 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String toString() { - return "SnapshotShardFailure{" + - "shardId=" + shardId + - ", reason='" + reason + '\'' + - ", nodeId='" + nodeId + '\'' + - ", status=" + status + - '}'; + return "SnapshotShardFailure{" + + "shardId=" + + shardId + + ", reason='" + + reason + + '\'' + + ", nodeId='" + + nodeId + + '\'' + + ", status=" + + status + + '}'; } - static final ConstructingObjectParser SNAPSHOT_SHARD_FAILURE_PARSER = - new ConstructingObjectParser<>("shard_failure", true, SnapshotShardFailure::constructSnapshotShardFailure); + static final ConstructingObjectParser SNAPSHOT_SHARD_FAILURE_PARSER = new ConstructingObjectParser<>( + "shard_failure", + true, + SnapshotShardFailure::constructSnapshotShardFailure + ); static { SNAPSHOT_SHARD_FAILURE_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("index")); @@ -171,10 +180,10 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; SnapshotShardFailure that = (SnapshotShardFailure) o; - return shardId.equals(that.shardId) && - Objects.equals(reason, that.reason) && - Objects.equals(nodeId, that.nodeId) && - status.getStatus() == that.status.getStatus(); + return shardId.equals(that.shardId) + && Objects.equals(reason, that.reason) + && Objects.equals(nodeId, that.nodeId) + && status.getStatus() == that.status.getStatus(); } @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardSizeInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardSizeInfo.java index 32629f19e1234..015133659d441 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardSizeInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardSizeInfo.java @@ -26,10 +26,15 @@ public Long getShardSize(ShardRouting shardRouting) { if (shardRouting.primary() && shardRouting.active() == false && shardRouting.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { - final RecoverySource.SnapshotRecoverySource snapshotRecoverySource = - (RecoverySource.SnapshotRecoverySource) shardRouting.recoverySource(); - return snapshotShardSizes.get(new InternalSnapshotsInfoService.SnapshotShard( - snapshotRecoverySource.snapshot(), snapshotRecoverySource.index(), shardRouting.shardId())); + final RecoverySource.SnapshotRecoverySource snapshotRecoverySource = (RecoverySource.SnapshotRecoverySource) shardRouting + .recoverySource(); + return snapshotShardSizes.get( + new InternalSnapshotsInfoService.SnapshotShard( + snapshotRecoverySource.snapshot(), + snapshotRecoverySource.index(), + shardRouting.shardId() + ) + ); } assert false : "Expected shard with snapshot recovery source but was " + shardRouting; return null; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 5a999c7caccc4..4d67c1b6c6355 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -83,8 +83,13 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements private final ResultDeduplicator remoteFailedRequestDeduplicator = new ResultDeduplicator<>(); - public SnapshotShardsService(Settings settings, ClusterService clusterService, RepositoriesService repositoriesService, - TransportService transportService, IndicesService indicesService) { + public SnapshotShardsService( + Settings settings, + ClusterService clusterService, + RepositoriesService repositoriesService, + TransportService transportService, + IndicesService indicesService + ) { this.indicesService = indicesService; this.repositoriesService = repositoriesService; this.transportService = transportService; @@ -97,12 +102,10 @@ public SnapshotShardsService(Settings settings, ClusterService clusterService, R } @Override - protected void doStart() { - } + protected void doStart() {} @Override - protected void doStop() { - } + protected void doStop() {} @Override protected void doClose() { @@ -140,8 +143,11 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh for (Map.Entry> snapshotShards : shardSnapshots.entrySet()) { Map shards = snapshotShards.getValue(); if (shards.containsKey(shardId)) { - logger.debug("[{}] shard closing, abort snapshotting for snapshot [{}]", - shardId, snapshotShards.getKey().getSnapshotId()); + logger.debug( + "[{}] shard closing, abort snapshotting for snapshot [{}]", + shardId, + snapshotShards.getKey().getSnapshotId() + ); shards.get(shardId).abortIfNotCompleted("shard is closing, aborting"); } } @@ -204,7 +210,7 @@ private void startNewSnapshots(SnapshotsInProgress snapshotsInProgress) { && snapshotShards.containsKey(shardId) == false) { logger.trace("[{}] adding shard to the queue", shardId); if (startedShards == null) { - startedShards = new HashMap<>(); + startedShards = new HashMap<>(); } startedShards.put(shardId, IndexShardSnapshotStatus.newInitializing(shardSnapshotStatus.generation())); } @@ -241,44 +247,50 @@ private void startNewShards(SnapshotsInProgress.Entry entry, Map() { - @Override - public void onResponse(ShardSnapshotResult shardSnapshotResult) { - final String newGeneration = shardSnapshotResult.getGeneration(); - assert newGeneration != null; - assert newGeneration.equals(snapshotStatus.generation()); - if (logger.isDebugEnabled()) { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); - logger.debug("[{}][{}] completed snapshot to [{}] with status [{}] at generation [{}]", - shardId, snapshot, snapshot.getRepository(), lastSnapshotStatus, snapshotStatus.generation()); - } - notifySuccessfulSnapshotShard(snapshot, shardId, shardSnapshotResult); + assert SnapshotsService.useShardGenerations(entry.version()) + || ShardGenerations.fixShardGeneration(snapshotStatus.generation()) == null + : "Found non-null, non-numeric shard generation [" + + snapshotStatus.generation() + + "] for snapshot with old-format compatibility"; + snapshot(shardId, snapshot, indexId, entry.userMetadata(), snapshotStatus, entry.version(), new ActionListener<>() { + @Override + public void onResponse(ShardSnapshotResult shardSnapshotResult) { + final String newGeneration = shardSnapshotResult.getGeneration(); + assert newGeneration != null; + assert newGeneration.equals(snapshotStatus.generation()); + if (logger.isDebugEnabled()) { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); + logger.debug( + "[{}][{}] completed snapshot to [{}] with status [{}] at generation [{}]", + shardId, + snapshot, + snapshot.getRepository(), + lastSnapshotStatus, + snapshotStatus.generation() + ); } + notifySuccessfulSnapshotShard(snapshot, shardId, shardSnapshotResult); + } - @Override - public void onFailure(Exception e) { - final String failure; - if (e instanceof AbortedSnapshotException) { - failure = "aborted"; - logger.debug(() -> new ParameterizedMessage("[{}][{}] aborted shard snapshot", shardId, snapshot), e); - } else { - failure = summarizeFailure(e); - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); - } - snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), failure); - notifyFailedSnapshotShard(snapshot, shardId, failure); + @Override + public void onFailure(Exception e) { + final String failure; + if (e instanceof AbortedSnapshotException) { + failure = "aborted"; + logger.debug(() -> new ParameterizedMessage("[{}][{}] aborted shard snapshot", shardId, snapshot), e); + } else { + failure = summarizeFailure(e); + logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); } - }); + snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), failure); + notifyFailedSnapshotShard(snapshot, shardId, failure); + } + }); } }); } - //package private for testing + // package private for testing static String summarizeFailure(Throwable t) { if (t.getCause() == null) { return t.getClass().getSimpleName() + "[" + t.getMessage() + "]"; @@ -306,8 +318,15 @@ static String summarizeFailure(Throwable t) { * @param snapshot snapshot * @param snapshotStatus snapshot status */ - private void snapshot(final ShardId shardId, final Snapshot snapshot, final IndexId indexId, final Map userMetadata, - final IndexShardSnapshotStatus snapshotStatus, Version version, ActionListener listener) { + private void snapshot( + final ShardId shardId, + final Snapshot snapshot, + final IndexId indexId, + final Map userMetadata, + final IndexShardSnapshotStatus snapshotStatus, + Version version, + ActionListener listener + ) { try { final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id()); if (indexShard.routingEntry().primary() == false) { @@ -328,9 +347,20 @@ private void snapshot(final ShardId shardId, final Snapshot snapshot, final Inde Engine.IndexCommitRef snapshotRef = null; try { snapshotRef = indexShard.acquireIndexCommitForSnapshot(); - repository.snapshotShard(new SnapshotShardContext(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), - indexId, snapshotRef, getShardStateId(indexShard, snapshotRef.getIndexCommit()), snapshotStatus, version, userMetadata, - listener)); + repository.snapshotShard( + new SnapshotShardContext( + indexShard.store(), + indexShard.mapperService(), + snapshot.getSnapshotId(), + indexId, + snapshotRef, + getShardStateId(indexShard, snapshotRef.getIndexCommit()), + snapshotStatus, + version, + userMetadata, + listener + ) + ); } catch (Exception e) { IOUtils.close(snapshotRef); throw e; @@ -361,8 +391,11 @@ private static String getShardStateId(IndexShard indexShard, IndexCommit snapsho if (maxSeqNo != seqNumInfo.localCheckpoint || maxSeqNo != indexShard.getLastSyncedGlobalCheckpoint()) { return null; } - return userCommitData.get(Engine.HISTORY_UUID_KEY) + "-" + - userCommitData.getOrDefault(Engine.FORCE_MERGE_UUID_KEY, "na") + "-" + maxSeqNo; + return userCommitData.get(Engine.HISTORY_UUID_KEY) + + "-" + + userCommitData.getOrDefault(Engine.FORCE_MERGE_UUID_KEY, "na") + + "-" + + maxSeqNo; } /** @@ -382,7 +415,7 @@ private void syncShardStatsOnNewMaster(ClusterChangedEvent event) { Map localShards = currentSnapshotShards(snapshot.snapshot()); if (localShards != null) { ImmutableOpenMap masterShards = snapshot.shards(); - for(Map.Entry localShard : localShards.entrySet()) { + for (Map.Entry localShard : localShards.entrySet()) { ShardId shardId = localShard.getKey(); ShardSnapshotStatus masterShard = masterShards.get(shardId); if (masterShard != null && masterShard.state().completed() == false) { @@ -391,14 +424,22 @@ private void syncShardStatsOnNewMaster(ClusterChangedEvent event) { // Master knows about the shard and thinks it has not completed if (stage == Stage.DONE) { // but we think the shard is done - we need to make new master know that the shard is done - logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, " + - "updating status on the master", snapshot.snapshot(), shardId); + logger.debug( + "[{}] new master thinks the shard [{}] is not completed but the shard is done locally, " + + "updating status on the master", + snapshot.snapshot(), + shardId + ); notifySuccessfulSnapshotShard(snapshot.snapshot(), shardId, localShard.getValue().getShardSnapshotResult()); } else if (stage == Stage.FAILURE) { // but we think the shard failed - we need to make new master know that the shard failed - logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, " + - "updating status on master", snapshot.snapshot(), shardId); + logger.debug( + "[{}] new master thinks the shard [{}] is not completed but the shard failed locally, " + + "updating status on master", + snapshot.snapshot(), + shardId + ); notifyFailedSnapshotShard(snapshot.snapshot(), shardId, indexShardSnapshotStatus.getFailure()); } } @@ -417,8 +458,11 @@ private void notifySuccessfulSnapshotShard(final Snapshot snapshot, final ShardI /** Notify the master node that the given shard failed to be snapshotted **/ private void notifyFailedSnapshotShard(final Snapshot snapshot, final ShardId shardId, final String failure) { - sendSnapshotShardUpdate(snapshot, shardId, - new ShardSnapshotStatus(clusterService.localNode().getId(), ShardState.FAILED, failure, null)); + sendSnapshotShardUpdate( + snapshot, + shardId, + new ShardSnapshotStatus(clusterService.localNode().getId(), ShardState.FAILED, failure, null) + ); } /** Updates the shard snapshot status by sending a {@link UpdateIndexShardSnapshotStatusRequest} to the master node */ @@ -434,11 +478,15 @@ public void onResponse(Void aVoid) { @Override public void onFailure(Exception e) { logger.warn( - () -> new ParameterizedMessage("[{}][{}] failed to update snapshot state to [{}]", shardId, snapshot, status), e); + () -> new ParameterizedMessage("[{}][{}] failed to update snapshot state to [{}]", shardId, snapshot, status), + e + ); } }, - (req, reqListener) -> transportService.sendRequest(transportService.getLocalNode(), - SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME, req, + (req, reqListener) -> transportService.sendRequest( + transportService.getLocalNode(), + SnapshotsService.UPDATE_SNAPSHOT_STATUS_ACTION_NAME, + req, new TransportResponseHandler() { @Override public ActionResponse.Empty read(StreamInput in) { @@ -454,7 +502,8 @@ public void handleResponse(ActionResponse.Empty response) { public void handleException(TransportException exp) { reqListener.onFailure(exp); } - }) + } + ) ); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotState.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotState.java index 869b5dff16aa2..ea93e8456126e 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotState.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotState.java @@ -8,7 +8,6 @@ package org.elasticsearch.snapshots; - /** * Represents the state that a snapshot can be in */ @@ -64,7 +63,6 @@ public boolean completed() { return completed; } - /** * Returns true if snapshot can be restored (at least partially) * @@ -97,4 +95,3 @@ public static SnapshotState fromValue(byte value) { } } } - diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 4f99b43b26487..1cac647edbb7f 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -151,7 +151,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus */ private final Map>> snapshotDeletionListeners = new HashMap<>(); - //Set of repositories currently running either a snapshot finalization or a snapshot delete. + // Set of repositories currently running either a snapshot finalization or a snapshot delete. private final Set currentlyFinalizing = Collections.synchronizedSet(new HashSet<>()); // Set of snapshots that are currently being ended by this node @@ -173,14 +173,25 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus * cluster state. The number of concurrent operations in a cluster state is defined as the sum of the sizes of * {@link SnapshotsInProgress#entries()} and {@link SnapshotDeletionsInProgress#getEntries()}. */ - public static final Setting MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING = - Setting.intSetting("snapshot.max_concurrent_operations", 1000, 1, Setting.Property.NodeScope, Setting.Property.Dynamic); + public static final Setting MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING = Setting.intSetting( + "snapshot.max_concurrent_operations", + 1000, + 1, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); private volatile int maxConcurrentOperations; - public SnapshotsService(Settings settings, ClusterService clusterService, IndexNameExpressionResolver indexNameExpressionResolver, - RepositoriesService repositoriesService, TransportService transportService, ActionFilters actionFilters, - Map systemIndexDescriptorMap) { + public SnapshotsService( + Settings settings, + ClusterService clusterService, + IndexNameExpressionResolver indexNameExpressionResolver, + RepositoriesService repositoriesService, + TransportService transportService, + ActionFilters actionFilters, + Map systemIndexDescriptorMap + ) { this.clusterService = clusterService; this.indexNameExpressionResolver = indexNameExpressionResolver; this.repositoriesService = repositoriesService; @@ -189,13 +200,18 @@ public SnapshotsService(Settings settings, ClusterService clusterService, IndexN // The constructor of UpdateSnapshotStatusAction will register itself to the TransportService. this.updateSnapshotStatusHandler = new UpdateSnapshotStatusAction( - transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver); + transportService, + clusterService, + threadPool, + actionFilters, + indexNameExpressionResolver + ); if (DiscoveryNode.isMasterNode(settings)) { // addLowPriorityApplier to make sure that Repository will be created before snapshot clusterService.addLowPriorityApplier(this); maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); - clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, - i -> maxConcurrentOperations = i); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); } this.systemIndexDescriptorMap = systemIndexDescriptorMap; } @@ -225,12 +241,11 @@ public void createSnapshot(final CreateSnapshotRequest request, final ActionList final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.snapshot()); validate(repositoryName, snapshotName); // TODO: create snapshot UUID in CreateSnapshotRequest and make this operation idempotent to cleanly deal with transport layer - // retries + // retries final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); // new UUID for the snapshot Repository repository = repositoriesService.repository(request.repository()); if (repository.isReadOnly()) { - listener.onFailure( - new RepositoryException(repository.getMetadata().name(), "cannot create snapshot in a readonly repository")); + listener.onFailure(new RepositoryException(repository.getMetadata().name(), "cannot create snapshot in a readonly repository")); return; } final Snapshot snapshot = new Snapshot(repositoryName, snapshotId); @@ -250,9 +265,15 @@ public void createSnapshot(final CreateSnapshotRequest request, final ActionList } else { // Otherwise, check for "none" then use the list of requested states if (requestedStates.contains(NO_FEATURE_STATES_VALUE)) { - listener.onFailure(new IllegalArgumentException("the feature_states value [" + - SnapshotsService.NO_FEATURE_STATES_VALUE + "] indicates that no feature states should be snapshotted, " + - "but other feature states were requested: " + requestedStates)); + listener.onFailure( + new IllegalArgumentException( + "the feature_states value [" + + SnapshotsService.NO_FEATURE_STATES_VALUE + + "] indicates that no feature states should be snapshotted, " + + "but other feature states were requested: " + + requestedStates + ) + ); return; } featureStatesSet = new HashSet<>(requestedStates); @@ -274,8 +295,10 @@ public ClusterState execute(ClusterState currentState) { final List runningSnapshots = snapshots.entries(); ensureSnapshotNameNotRunning(runningSnapshots, repositoryName, snapshotName); validate(repositoryName, snapshotName, currentState); - final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); ensureNoCleanupInProgress(currentState, repositoryName, snapshotName); ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); // Store newSnapshot here to be processed in clusterStateProcessed @@ -289,12 +312,18 @@ public ClusterState execute(ClusterState currentState) { } else { final Set indexNames = new HashSet<>(indices); featureStates = featureStatesSet.stream() - .map(feature -> new SnapshotFeatureInfo(feature, - systemIndexDescriptorMap.get(feature).getIndexDescriptors().stream() - .flatMap(descriptor -> descriptor.getMatchingIndices(currentState.metadata()).stream()) - .collect(Collectors.toList()))) - .filter(featureInfo -> featureInfo.getIndices().isEmpty() == false) // Omit any empty featureStates - .collect(Collectors.toList()); + .map( + feature -> new SnapshotFeatureInfo( + feature, + systemIndexDescriptorMap.get(feature) + .getIndexDescriptors() + .stream() + .flatMap(descriptor -> descriptor.getMatchingIndices(currentState.metadata()).stream()) + .collect(Collectors.toList()) + ) + ) + .filter(featureInfo -> featureInfo.getIndices().isEmpty() == false) // Omit any empty featureStates + .collect(Collectors.toList()); for (SnapshotFeatureInfo featureState : featureStates) { indexNames.addAll(featureState.getIndices()); } @@ -308,16 +337,29 @@ public ClusterState execute(ClusterState currentState) { indices = List.copyOf(indexNames); } - final List dataStreams = - indexNameExpressionResolver.dataStreamNames(currentState, request.indicesOptions(), request.indices()); + final List dataStreams = indexNameExpressionResolver.dataStreamNames( + currentState, + request.indicesOptions(), + request.indices() + ); logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices); - final Map indexIds = - repositoryData.resolveNewIndices(indices, getInFlightIndexIds(runningSnapshots, repositoryName)); + final Map indexIds = repositoryData.resolveNewIndices( + indices, + getInFlightIndexIds(runningSnapshots, repositoryName) + ); final Version version = minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null); - ImmutableOpenMap shards = shards(snapshots, deletionsInProgress, currentState.metadata(), - currentState.routingTable(), indexIds.values(), useShardGenerations(version), repositoryData, repositoryName); + ImmutableOpenMap shards = shards( + snapshots, + deletionsInProgress, + currentState.metadata(), + currentState.routingTable(), + indexIds.values(), + useShardGenerations(version), + repositoryData, + repositoryName + ); if (request.partial() == false) { Set missing = new HashSet<>(); for (ObjectObjectCursor entry : shards) { @@ -327,15 +369,27 @@ public ClusterState execute(ClusterState currentState) { } if (missing.isEmpty() == false) { throw new SnapshotException( - new Snapshot(repositoryName, snapshotId), "Indices don't have primary shards " + missing); + new Snapshot(repositoryName, snapshotId), + "Indices don't have primary shards " + missing + ); } } newEntry = SnapshotsInProgress.startedEntry( - new Snapshot(repositoryName, snapshotId), request.includeGlobalState(), request.partial(), - indexIds, dataStreams, threadPool.absoluteTimeInMillis(), repositoryData.getGenId(), shards, - userMeta, version, featureStates); - return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, - SnapshotsInProgress.of(CollectionUtils.appendToCopy(runningSnapshots, newEntry))).build(); + new Snapshot(repositoryName, snapshotId), + request.includeGlobalState(), + request.partial(), + indexIds, + dataStreams, + threadPool.absoluteTimeInMillis(), + repositoryData.getGenId(), + shards, + userMeta, + version, + featureStates + ); + return ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(CollectionUtils.appendToCopy(runningSnapshots, newEntry))) + .build(); } @Override @@ -358,8 +412,11 @@ public void clusterStateProcessed(String source, ClusterState oldState, final Cl }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } - private static void ensureSnapshotNameNotRunning(List runningSnapshots, String repositoryName, - String snapshotName) { + private static void ensureSnapshotNameNotRunning( + List runningSnapshots, + String repositoryName, + String snapshotName + ) { if (runningSnapshots.stream().anyMatch(s -> { final Snapshot running = s.snapshot(); return running.getRepository().equals(repositoryName) && running.getSnapshotId().getName().equals(snapshotName); @@ -379,7 +436,7 @@ private static Map getInFlightIndexIds(List listener) { final String repositoryName = request.repository(); Repository repository = repositoriesService.repository(repositoryName); @@ -390,7 +447,7 @@ public void cloneSnapshot(CloneSnapshotRequest request, ActionListener lis final String snapshotName = indexNameExpressionResolver.resolveDateMathExpression(request.target()); validate(repositoryName, snapshotName); // TODO: create snapshot UUID in CloneSnapshotRequest and make this operation idempotent to cleanly deal with transport layer - // retries + // retries final SnapshotId snapshotId = new SnapshotId(snapshotName, UUIDs.randomBase64UUID()); final Snapshot snapshot = new Snapshot(repositoryName, snapshotId); initializingClones.add(snapshot); @@ -408,15 +465,20 @@ public ClusterState execute(ClusterState currentState) { validate(repositoryName, snapshotName, currentState); final SnapshotId sourceSnapshotId = repositoryData.getSnapshotIds() - .stream() - .filter(src -> src.getName().equals(request.source())) - .findAny() - .orElseThrow(() -> new SnapshotMissingException(repositoryName, request.source())); - final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + .stream() + .filter(src -> src.getName().equals(request.source())) + .findAny() + .orElseThrow(() -> new SnapshotMissingException(repositoryName, request.source())); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(sourceSnapshotId))) { - throw new ConcurrentSnapshotExecutionException(repositoryName, sourceSnapshotId.getName(), - "cannot clone from snapshot that is being deleted"); + throw new ConcurrentSnapshotExecutionException( + repositoryName, + sourceSnapshotId.getName(), + "cannot clone from snapshot that is being deleted" + ); } ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); final List indicesForSnapshot = new ArrayList<>(); @@ -425,20 +487,32 @@ public ClusterState execute(ClusterState currentState) { indicesForSnapshot.add(indexId.getName()); } } - final List matchingIndices = - SnapshotUtils.filterIndices(indicesForSnapshot, request.indices(), request.indicesOptions()); + final List matchingIndices = SnapshotUtils.filterIndices( + indicesForSnapshot, + request.indices(), + request.indicesOptions() + ); if (matchingIndices.isEmpty()) { - throw new SnapshotException(new Snapshot(repositoryName, sourceSnapshotId), - "No indices in the source snapshot [" + sourceSnapshotId + "] matched requested pattern [" - + Strings.arrayToCommaDelimitedString(request.indices()) + "]"); + throw new SnapshotException( + new Snapshot(repositoryName, sourceSnapshotId), + "No indices in the source snapshot [" + + sourceSnapshotId + + "] matched requested pattern [" + + Strings.arrayToCommaDelimitedString(request.indices()) + + "]" + ); } newEntry = SnapshotsInProgress.startClone( - snapshot, sourceSnapshotId, - repositoryData.resolveIndices(matchingIndices), - threadPool.absoluteTimeInMillis(), repositoryData.getGenId(), - minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null)); - return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, - SnapshotsInProgress.of(CollectionUtils.appendToCopy(runningSnapshots, newEntry))).build(); + snapshot, + sourceSnapshotId, + repositoryData.resolveIndices(matchingIndices), + threadPool.absoluteTimeInMillis(), + repositoryData.getGenId(), + minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null) + ); + return ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(CollectionUtils.appendToCopy(runningSnapshots, newEntry))) + .build(); } @Override @@ -458,11 +532,16 @@ public void clusterStateProcessed(String source, ClusterState oldState, final Cl } private static void ensureNoCleanupInProgress(ClusterState currentState, String repositoryName, String snapshotName) { - final RepositoryCleanupInProgress repositoryCleanupInProgress = - currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress.EMPTY + ); if (repositoryCleanupInProgress.hasCleanupInProgress()) { - throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, - "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]"); + throw new ConcurrentSnapshotExecutionException( + repositoryName, + snapshotName, + "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" + ); } } @@ -470,7 +549,10 @@ private static void ensureSnapshotNameAvailableInRepo(RepositoryData repositoryD // check if the snapshot name already exists in the repository if (repositoryData.getSnapshotIds().stream().anyMatch(s -> s.getName().equals(snapshotName))) { throw new InvalidSnapshotNameException( - repository.getMetadata().name(), snapshotName, "snapshot with the same name already exists"); + repository.getMetadata().name(), + snapshotName, + "snapshot with the same name already exists" + ); } } @@ -499,13 +581,17 @@ private void startCloning(Repository repository, SnapshotsInProgress.Entry clone repository.getSnapshotInfo(sourceSnapshot, snapshotInfoListener); final StepListener>> allShardCountsListener = new StepListener<>(); - final GroupedActionListener> shardCountListener = - new GroupedActionListener<>(allShardCountsListener, indices.size()); + final GroupedActionListener> shardCountListener = new GroupedActionListener<>( + allShardCountsListener, + indices.size() + ); snapshotInfoListener.whenComplete(snapshotInfo -> { for (IndexId indexId : indices) { if (RestoreService.failed(snapshotInfo, indexId.getName())) { - throw new SnapshotException(targetSnapshot, "Can't clone index [" + indexId + - "] because its snapshot was not successful."); + throw new SnapshotException( + targetSnapshot, + "Can't clone index [" + indexId + "] because its snapshot was not successful." + ); } } // 2. step, load the number of shards we have in each index to be cloned from the index metadata. @@ -526,8 +612,7 @@ private void startCloning(Repository repository, SnapshotsInProgress.Entry clone @Override public ClusterState execute(ClusterState currentState) { - final SnapshotsInProgress snapshotsInProgress = - currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List updatedEntries = new ArrayList<>(snapshotsInProgress.entries()); boolean changed = false; final String localNodeId = currentState.nodes().getLocalNodeId(); @@ -540,10 +625,11 @@ public ClusterState execute(ClusterState currentState) { continue; } if (cloneEntry.snapshot().getSnapshotId().equals(entry.snapshot().getSnapshotId())) { - final ImmutableOpenMap.Builder clonesBuilder = - ImmutableOpenMap.builder(); + final ImmutableOpenMap.Builder clonesBuilder = ImmutableOpenMap.builder(); final boolean readyToExecute = currentState.custom( - SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).hasExecutingDeletion(repoName) == false; + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ).hasExecutingDeletion(repoName) == false; final InFlightShardSnapshotStates inFlightShardStates; if (readyToExecute) { inFlightShardStates = InFlightShardSnapshotStates.forRepo(repoName, snapshotsInProgress.entries()); @@ -558,8 +644,13 @@ public ClusterState execute(ClusterState currentState) { if (readyToExecute == false || inFlightShardStates.isActive(indexName, shardId)) { clonesBuilder.put(repoShardId, ShardSnapshotStatus.UNASSIGNED_QUEUED); } else { - clonesBuilder.put(repoShardId, new ShardSnapshotStatus(localNodeId, - inFlightShardStates.generationForShard(repoShardId.index(), shardId, shardGenerations))); + clonesBuilder.put( + repoShardId, + new ShardSnapshotStatus( + localNodeId, + inFlightShardStates.generationForShard(repoShardId.index(), shardId, shardGenerations) + ) + ); } } } @@ -608,45 +699,86 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS private final Set currentlyCloning = Collections.synchronizedSet(new HashSet<>()); - private void runReadyClone(Snapshot target, SnapshotId sourceSnapshot, ShardSnapshotStatus shardStatusBefore, - RepositoryShardId repoShardId, Repository repository) { + private void runReadyClone( + Snapshot target, + SnapshotId sourceSnapshot, + ShardSnapshotStatus shardStatusBefore, + RepositoryShardId repoShardId, + Repository repository + ) { final SnapshotId targetSnapshot = target.getSnapshotId(); final String localNodeId = clusterService.localNode().getId(); if (currentlyCloning.add(repoShardId)) { - repository.cloneShardSnapshot(sourceSnapshot, targetSnapshot, repoShardId, shardStatusBefore.generation(), ActionListener.wrap( + repository.cloneShardSnapshot( + sourceSnapshot, + targetSnapshot, + repoShardId, + shardStatusBefore.generation(), + ActionListener.wrap( shardSnapshotResult -> innerUpdateSnapshotState( - new ShardSnapshotUpdate(target, repoShardId, ShardSnapshotStatus.success(localNodeId, shardSnapshotResult)), - ActionListener.runBefore( - ActionListener.wrap( - v -> logger.trace("Marked [{}] as successfully cloned from [{}] to [{}]", repoShardId, - sourceSnapshot, targetSnapshot), - e -> { - logger.warn("Cluster state update after successful shard clone [{}] failed", repoShardId); - failAllListenersOnMasterFailOver(e); - } - ), () -> currentlyCloning.remove(repoShardId)) - ), e -> innerUpdateSnapshotState( - new ShardSnapshotUpdate(target, repoShardId, - new ShardSnapshotStatus(localNodeId, ShardState.FAILED, "failed to clone shard snapshot", null)), - ActionListener.runBefore(ActionListener.wrap( - v -> logger.trace("Marked [{}] as failed clone from [{}] to [{}]", repoShardId, - sourceSnapshot, targetSnapshot), - ex -> { - logger.warn("Cluster state update after failed shard clone [{}] failed", repoShardId); - failAllListenersOnMasterFailOver(ex); - } - ), () -> currentlyCloning.remove(repoShardId))))); + new ShardSnapshotUpdate(target, repoShardId, ShardSnapshotStatus.success(localNodeId, shardSnapshotResult)), + ActionListener.runBefore( + ActionListener.wrap( + v -> logger.trace( + "Marked [{}] as successfully cloned from [{}] to [{}]", + repoShardId, + sourceSnapshot, + targetSnapshot + ), + e -> { + logger.warn("Cluster state update after successful shard clone [{}] failed", repoShardId); + failAllListenersOnMasterFailOver(e); + } + ), + () -> currentlyCloning.remove(repoShardId) + ) + ), + e -> innerUpdateSnapshotState( + new ShardSnapshotUpdate( + target, + repoShardId, + new ShardSnapshotStatus(localNodeId, ShardState.FAILED, "failed to clone shard snapshot", null) + ), + ActionListener.runBefore( + ActionListener.wrap( + v -> logger.trace( + "Marked [{}] as failed clone from [{}] to [{}]", + repoShardId, + sourceSnapshot, + targetSnapshot + ), + ex -> { + logger.warn("Cluster state update after failed shard clone [{}] failed", repoShardId); + failAllListenersOnMasterFailOver(ex); + } + ), + () -> currentlyCloning.remove(repoShardId) + ) + ) + ) + ); } } - private void ensureBelowConcurrencyLimit(String repository, String name, SnapshotsInProgress snapshotsInProgress, - SnapshotDeletionsInProgress deletionsInProgress) { + private void ensureBelowConcurrencyLimit( + String repository, + String name, + SnapshotsInProgress snapshotsInProgress, + SnapshotDeletionsInProgress deletionsInProgress + ) { final int inProgressOperations = snapshotsInProgress.entries().size() + deletionsInProgress.getEntries().size(); final int maxOps = maxConcurrentOperations; if (inProgressOperations >= maxOps) { - throw new ConcurrentSnapshotExecutionException(repository, name, - "Cannot start another operation, already running [" + inProgressOperations + "] operations and the current" + - " limit for concurrent snapshot operations is set to [" + maxOps + "]"); + throw new ConcurrentSnapshotExecutionException( + repository, + name, + "Cannot start another operation, already running [" + + inProgressOperations + + "] operations and the current" + + " limit for concurrent snapshot operations is set to [" + + maxOps + + "]" + ); } } @@ -685,9 +817,11 @@ private static void validate(final String repositoryName, final String snapshotN throw new InvalidSnapshotNameException(repositoryName, snapshotName, "must be lowercase"); } if (Strings.validFileName(snapshotName) == false) { - throw new InvalidSnapshotNameException(repositoryName, - snapshotName, - "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS); + throw new InvalidSnapshotNameException( + repositoryName, + snapshotName, + "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS + ); } } @@ -701,8 +835,8 @@ private static ShardGenerations buildGenerations(SnapshotsInProgress.Entry snaps } else { snapshot.shards().forEach(c -> { if (metadata.index(c.key.getIndex()) == null) { - assert snapshot.partial() : - "Index [" + c.key.getIndex() + "] was deleted during a snapshot but snapshot was not partial."; + assert snapshot.partial() + : "Index [" + c.key.getIndex() + "] was deleted during a snapshot but snapshot was not partial."; return; } final IndexId indexId = snapshot.indices().get(c.key.getIndexName()); @@ -737,8 +871,8 @@ private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, for (String dataStreamName : snapshot.dataStreams()) { DataStream dataStream = metadata.dataStreams().get(dataStreamName); if (dataStream == null) { - assert snapshot.partial() : "Data stream [" + dataStreamName + - "] was deleted during a snapshot but snapshot was not partial."; + assert snapshot.partial() + : "Data stream [" + dataStreamName + "] was deleted during a snapshot but snapshot was not partial."; } else { boolean missingIndex = false; for (Index index : dataStream.getIndices()) { @@ -768,8 +902,11 @@ private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, * @param snapshots list of snapshots that will be used as a filter, empty list means no snapshots are filtered * @return list of metadata for currently running snapshots */ - public static List currentSnapshots(@Nullable SnapshotsInProgress snapshotsInProgress, String repository, - List snapshots) { + public static List currentSnapshots( + @Nullable SnapshotsInProgress snapshotsInProgress, + String repository, + List snapshots + ) { if (snapshotsInProgress == null || snapshotsInProgress.entries().isEmpty()) { return Collections.emptyList(); } @@ -820,8 +957,10 @@ public void applyClusterState(ClusterChangedEvent event) { // We don't remove old master when master flips anymore. So, we need to check for change in master SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final boolean newMaster = event.previousState().nodes().isLocalNodeElectedMaster() == false; - processExternalChanges(newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), - event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event)); + processExternalChanges( + newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), + event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event) + ); } else if (snapshotCompletionListeners.isEmpty() == false) { // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster @@ -845,25 +984,30 @@ private boolean assertConsistentWithClusterState(ClusterState state) { if (snapshotsInProgress.entries().isEmpty() == false) { synchronized (endingSnapshots) { final Set runningSnapshots = Stream.concat( - snapshotsInProgress.entries().stream().map(SnapshotsInProgress.Entry::snapshot), - endingSnapshots.stream()) - .collect(Collectors.toSet()); + snapshotsInProgress.entries().stream().map(SnapshotsInProgress.Entry::snapshot), + endingSnapshots.stream() + ).collect(Collectors.toSet()); final Set snapshotListenerKeys = snapshotCompletionListeners.keySet(); - assert runningSnapshots.containsAll(snapshotListenerKeys) : "Saw completion listeners for unknown snapshots in " - + snapshotListenerKeys + " but running snapshots are " + runningSnapshots; + assert runningSnapshots.containsAll(snapshotListenerKeys) + : "Saw completion listeners for unknown snapshots in " + + snapshotListenerKeys + + " but running snapshots are " + + runningSnapshots; } } - final SnapshotDeletionsInProgress snapshotDeletionsInProgress = - state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); if (snapshotDeletionsInProgress.hasDeletionsInProgress()) { synchronized (repositoryOperations.runningDeletions) { final Set runningDeletes = Stream.concat( - snapshotDeletionsInProgress.getEntries().stream().map(SnapshotDeletionsInProgress.Entry::uuid), - repositoryOperations.runningDeletions.stream()) - .collect(Collectors.toSet()); + snapshotDeletionsInProgress.getEntries().stream().map(SnapshotDeletionsInProgress.Entry::uuid), + repositoryOperations.runningDeletions.stream() + ).collect(Collectors.toSet()); final Set deleteListenerKeys = snapshotDeletionListeners.keySet(); - assert runningDeletes.containsAll(deleteListenerKeys) : "Saw deletions listeners for unknown uuids in " - + deleteListenerKeys + " but running deletes are " + runningDeletes; + assert runningDeletes.containsAll(deleteListenerKeys) + : "Saw deletions listeners for unknown uuids in " + deleteListenerKeys + " but running deletes are " + runningDeletes; } } return true; @@ -873,23 +1017,25 @@ private boolean assertConsistentWithClusterState(ClusterState state) { // to be assigned private static boolean assertNoDanglingSnapshots(ClusterState state) { final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - final SnapshotDeletionsInProgress snapshotDeletionsInProgress = - state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - final Set reposWithRunningDelete = snapshotDeletionsInProgress.getEntries().stream() - .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED) - .map(SnapshotDeletionsInProgress.Entry::repository).collect(Collectors.toSet()); + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); + final Set reposWithRunningDelete = snapshotDeletionsInProgress.getEntries() + .stream() + .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED) + .map(SnapshotDeletionsInProgress.Entry::repository) + .collect(Collectors.toSet()); final Set reposSeen = new HashSet<>(); for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { if (reposSeen.add(entry.repository())) { for (ObjectCursor value : (entry.isClone() ? entry.clones() : entry.shards()).values()) { if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { assert reposWithRunningDelete.contains(entry.repository()) - : "Found shard snapshot waiting to be assigned in [" + entry + - "] but it is not blocked by any running delete"; + : "Found shard snapshot waiting to be assigned in [" + entry + "] but it is not blocked by any running delete"; } else if (value.value.isActive()) { assert reposWithRunningDelete.contains(entry.repository()) == false - : "Found shard snapshot actively executing in [" + entry + - "] when it should be blocked by a running delete"; + : "Found shard snapshot actively executing in [" + entry + "] when it should be blocked by a running delete"; } } } @@ -911,123 +1057,151 @@ private void processExternalChanges(boolean changedNodes, boolean startShards) { // nothing to do, no relevant external change happened return; } - clusterService.submitStateUpdateTask("update snapshot after shards started [" + startShards + - "] or node configuration changed [" + changedNodes + "]", new ClusterStateUpdateTask() { + clusterService.submitStateUpdateTask( + "update snapshot after shards started [" + startShards + "] or node configuration changed [" + changedNodes + "]", + new ClusterStateUpdateTask() { - private final Collection finishedSnapshots = new ArrayList<>(); + private final Collection finishedSnapshots = new ArrayList<>(); - private final Collection deletionsToExecute = new ArrayList<>(); + private final Collection deletionsToExecute = new ArrayList<>(); - @Override - public ClusterState execute(ClusterState currentState) { - RoutingTable routingTable = currentState.routingTable(); - final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - DiscoveryNodes nodes = currentState.nodes(); - boolean changed = false; - final EnumSet statesToUpdate; - // If we are reacting to a change in the cluster node configuration we have to update the shard states of both started and - // aborted snapshots to potentially fail shards running on the removed nodes - if (changedNodes) { - statesToUpdate = EnumSet.of(State.STARTED, State.ABORTED); - } else { - // We are reacting to shards that started only so which only affects the individual shard states of started snapshots - statesToUpdate = EnumSet.of(State.STARTED); - } - ArrayList updatedSnapshotEntries = new ArrayList<>(); - - // We keep a cache of shards that failed in this map. If we fail a shardId for a given repository because of - // a node leaving or shard becoming unassigned for one snapshot, we will also fail it for all subsequent enqueued snapshots - // for the same repository - final Map> knownFailures = new HashMap<>(); - - for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { - if (statesToUpdate.contains(snapshot.state())) { - // Currently initializing clone - if (snapshot.isClone() && snapshot.clones().isEmpty()) { - if (initializingClones.contains(snapshot.snapshot())) { - updatedSnapshotEntries.add(snapshot); + @Override + public ClusterState execute(ClusterState currentState) { + RoutingTable routingTable = currentState.routingTable(); + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + DiscoveryNodes nodes = currentState.nodes(); + boolean changed = false; + final EnumSet statesToUpdate; + // If we are reacting to a change in the cluster node configuration we have to update the shard states of both started + // and + // aborted snapshots to potentially fail shards running on the removed nodes + if (changedNodes) { + statesToUpdate = EnumSet.of(State.STARTED, State.ABORTED); + } else { + // We are reacting to shards that started only so which only affects the individual shard states of started + // snapshots + statesToUpdate = EnumSet.of(State.STARTED); + } + ArrayList updatedSnapshotEntries = new ArrayList<>(); + + // We keep a cache of shards that failed in this map. If we fail a shardId for a given repository because of + // a node leaving or shard becoming unassigned for one snapshot, we will also fail it for all subsequent enqueued + // snapshots + // for the same repository + final Map> knownFailures = new HashMap<>(); + + for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { + if (statesToUpdate.contains(snapshot.state())) { + // Currently initializing clone + if (snapshot.isClone() && snapshot.clones().isEmpty()) { + if (initializingClones.contains(snapshot.snapshot())) { + updatedSnapshotEntries.add(snapshot); + } else { + logger.debug("removing not yet start clone operation [{}]", snapshot); + changed = true; + } } else { - logger.debug("removing not yet start clone operation [{}]", snapshot); - changed = true; + ImmutableOpenMap shards = processWaitingShardsAndRemovedNodes( + snapshot.shards(), + routingTable, + nodes, + knownFailures.computeIfAbsent(snapshot.repository(), k -> new HashMap<>()) + ); + if (shards != null) { + final SnapshotsInProgress.Entry updatedSnapshot = snapshot.withShardStates(shards); + changed = true; + if (updatedSnapshot.state().completed()) { + finishedSnapshots.add(updatedSnapshot); + } + updatedSnapshotEntries.add(updatedSnapshot); + } else { + updatedSnapshotEntries.add(snapshot); + } } + } else if (snapshot.repositoryStateId() == RepositoryData.UNKNOWN_REPO_GEN) { + // BwC path, older versions could create entries with unknown repo GEN in INIT or ABORTED state that did not yet + // write anything to the repository physically. This means we can simply remove these from the cluster state + // without having to do any additional cleanup. + changed = true; + logger.debug("[{}] was found in dangling INIT or ABORTED state", snapshot); } else { - ImmutableOpenMap shards = processWaitingShardsAndRemovedNodes(snapshot.shards(), - routingTable, nodes, knownFailures.computeIfAbsent(snapshot.repository(), k -> new HashMap<>())); - if (shards != null) { - final SnapshotsInProgress.Entry updatedSnapshot = snapshot.withShardStates(shards); - changed = true; - if (updatedSnapshot.state().completed()) { - finishedSnapshots.add(updatedSnapshot); - } - updatedSnapshotEntries.add(updatedSnapshot); - } else { - updatedSnapshotEntries.add(snapshot); + if (snapshot.state().completed() || completed(snapshot.shards().values())) { + finishedSnapshots.add(snapshot); } + updatedSnapshotEntries.add(snapshot); } - } else if (snapshot.repositoryStateId() == RepositoryData.UNKNOWN_REPO_GEN) { - // BwC path, older versions could create entries with unknown repo GEN in INIT or ABORTED state that did not yet - // write anything to the repository physically. This means we can simply remove these from the cluster state - // without having to do any additional cleanup. - changed = true; - logger.debug("[{}] was found in dangling INIT or ABORTED state", snapshot); - } else { - if (snapshot.state().completed() || completed(snapshot.shards().values())) { - finishedSnapshots.add(snapshot); - } - updatedSnapshotEntries.add(snapshot); } - } - final ClusterState res = readyDeletions( - changed ? ClusterState.builder(currentState).putCustom( - SnapshotsInProgress.TYPE, SnapshotsInProgress.of(updatedSnapshotEntries)).build() : - currentState).v1(); - for (SnapshotDeletionsInProgress.Entry delete - : res.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { - if (delete.state() == SnapshotDeletionsInProgress.State.STARTED) { - deletionsToExecute.add(delete); + final ClusterState res = readyDeletions( + changed + ? ClusterState.builder(currentState) + .putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(updatedSnapshotEntries)) + .build() + : currentState + ).v1(); + for (SnapshotDeletionsInProgress.Entry delete : res.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ).getEntries()) { + if (delete.state() == SnapshotDeletionsInProgress.State.STARTED) { + deletionsToExecute.add(delete); + } } + return res; } - return res; - } - @Override - public void onFailure(String source, Exception e) { - logger.warn(() -> new ParameterizedMessage( - "failed to update snapshot state after shards started or nodes removed from [{}] ", source), e); - } + @Override + public void onFailure(String source, Exception e) { + logger.warn( + () -> new ParameterizedMessage( + "failed to update snapshot state after shards started or nodes removed from [{}] ", + source + ), + e + ); + } - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - final SnapshotDeletionsInProgress snapshotDeletionsInProgress = - newState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - if (finishedSnapshots.isEmpty() == false) { - // If we found snapshots that should be finalized as a result of the CS update we try to initiate finalization for them - // unless there is an executing snapshot delete already. If there is an executing snapshot delete we don't have to - // enqueue the snapshot finalizations here because the ongoing delete will take care of that when removing the delete - // from the cluster state - final Set reposWithRunningDeletes = snapshotDeletionsInProgress.getEntries().stream() + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = newState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); + if (finishedSnapshots.isEmpty() == false) { + // If we found snapshots that should be finalized as a result of the CS update we try to initiate finalization for + // them + // unless there is an executing snapshot delete already. If there is an executing snapshot delete we don't have to + // enqueue the snapshot finalizations here because the ongoing delete will take care of that when removing the + // delete + // from the cluster state + final Set reposWithRunningDeletes = snapshotDeletionsInProgress.getEntries() + .stream() .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED) - .map(SnapshotDeletionsInProgress.Entry::repository).collect(Collectors.toSet()); - for (SnapshotsInProgress.Entry entry : finishedSnapshots) { - if (reposWithRunningDeletes.contains(entry.repository()) == false) { - endSnapshot(entry, newState.metadata(), null); + .map(SnapshotDeletionsInProgress.Entry::repository) + .collect(Collectors.toSet()); + for (SnapshotsInProgress.Entry entry : finishedSnapshots) { + if (reposWithRunningDeletes.contains(entry.repository()) == false) { + endSnapshot(entry, newState.metadata(), null); + } } } - } - startExecutableClones(newState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), null); - // run newly ready deletes - for (SnapshotDeletionsInProgress.Entry entry : deletionsToExecute) { - if (tryEnterRepoLoop(entry.repository())) { - deleteSnapshotsFromRepository(entry, newState.nodes().getMinNodeVersion()); + startExecutableClones(newState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), null); + // run newly ready deletes + for (SnapshotDeletionsInProgress.Entry entry : deletionsToExecute) { + if (tryEnterRepoLoop(entry.repository())) { + deleteSnapshotsFromRepository(entry, newState.nodes().getMinNodeVersion()); + } } } } - }); + ); } private static ImmutableOpenMap processWaitingShardsAndRemovedNodes( - ImmutableOpenMap snapshotShards, RoutingTable routingTable, DiscoveryNodes nodes, - Map knownFailures) { + ImmutableOpenMap snapshotShards, + RoutingTable routingTable, + DiscoveryNodes nodes, + Map knownFailures + ) { boolean snapshotChanged = false; ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); for (ObjectObjectCursor shardEntry : snapshotShards) { @@ -1054,8 +1228,10 @@ private static ImmutableOpenMap processWaitingShar // Shard that we were waiting for has started on a node, let's process it snapshotChanged = true; logger.trace("starting shard that we were waiting for [{}] on node [{}]", shardId, shardStatus.nodeId()); - shards.put(shardId, - new ShardSnapshotStatus(shardRouting.primaryShard().currentNodeId(), shardStatus.generation())); + shards.put( + shardId, + new ShardSnapshotStatus(shardRouting.primaryShard().currentNodeId(), shardStatus.generation()) + ); continue; } else if (shardRouting.primaryShard().initializing() || shardRouting.primaryShard().relocating()) { // Shard that we were waiting for hasn't started yet or still relocating - will continue to wait @@ -1067,8 +1243,12 @@ private static ImmutableOpenMap processWaitingShar // Shard that we were waiting for went into unassigned state or disappeared - giving up snapshotChanged = true; logger.warn("failing snapshot of shard [{}] on unassigned shard [{}]", shardId, shardStatus.nodeId()); - final ShardSnapshotStatus failedState = new ShardSnapshotStatus(shardStatus.nodeId(), ShardState.FAILED, - "shard is unassigned", shardStatus.generation()); + final ShardSnapshotStatus failedState = new ShardSnapshotStatus( + shardStatus.nodeId(), + ShardState.FAILED, + "shard is unassigned", + shardStatus.generation() + ); shards.put(shardId, failedState); knownFailures.put(shardId, failedState); } else if (shardStatus.state().completed() == false && shardStatus.nodeId() != null) { @@ -1077,10 +1257,13 @@ private static ImmutableOpenMap processWaitingShar } else { // TODO: Restart snapshot on another node? snapshotChanged = true; - logger.warn("failing snapshot of shard [{}] on closed node [{}]", - shardId, shardStatus.nodeId()); - final ShardSnapshotStatus failedState = new ShardSnapshotStatus(shardStatus.nodeId(), ShardState.FAILED, - "node shutdown", shardStatus.generation()); + logger.warn("failing snapshot of shard [{}] on closed node [{}]", shardId, shardStatus.nodeId()); + final ShardSnapshotStatus failedState = new ShardSnapshotStatus( + shardStatus.nodeId(), + ShardState.FAILED, + "node shutdown", + shardStatus.generation() + ); shards.put(shardId, failedState); knownFailures.put(shardId, failedState); } @@ -1104,8 +1287,7 @@ private static boolean waitingShardsStartedOrUnassigned(SnapshotsInProgress snap } final ShardId shardId = shardStatus.key; if (event.indexRoutingTableChanged(shardId.getIndexName())) { - IndexRoutingTable indexShardRoutingTable = - event.state().getRoutingTable().index(shardId.getIndex()); + IndexRoutingTable indexShardRoutingTable = event.state().getRoutingTable().index(shardId.getIndex()); if (indexShardRoutingTable == null) { // index got removed concurrently and we have to fail WAITING state shards return true; @@ -1127,21 +1309,20 @@ private static boolean removedNodesCleanupNeeded(SnapshotsInProgress snapshotsIn return false; } final Set removedNodeIds = removedNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toSet()); - return snapshotsInProgress.entries().stream() - .anyMatch(snapshot -> { - if (snapshot.state().completed()) { - // nothing to do for already completed snapshots - return false; - } - for (ObjectCursor shardStatus : snapshot.shards().values()) { - final ShardSnapshotStatus shardSnapshotStatus = shardStatus.value; - if (shardSnapshotStatus.state().completed() == false && removedNodeIds.contains(shardSnapshotStatus.nodeId())) { - // Snapshot had an incomplete shard running on a removed node so we need to adjust that shard's snapshot status - return true; - } - } - return false; - }); + return snapshotsInProgress.entries().stream().anyMatch(snapshot -> { + if (snapshot.state().completed()) { + // nothing to do for already completed snapshots + return false; + } + for (ObjectCursor shardStatus : snapshot.shards().values()) { + final ShardSnapshotStatus shardSnapshotStatus = shardStatus.value; + if (shardSnapshotStatus.state().completed() == false && removedNodeIds.contains(shardSnapshotStatus.nodeId())) { + // Snapshot had an incomplete shard running on a removed node so we need to adjust that shard's snapshot status + return true; + } + } + return false; + }); } /** @@ -1167,8 +1348,10 @@ public void onResponse(RepositoryData repositoryData) { @Override public void onFailure(Exception e) { - clusterService.submitStateUpdateTask("fail repo tasks for [" + repoName + "]", - new FailPendingRepoTasksTask(repoName, e)); + clusterService.submitStateUpdateTask( + "fail repo tasks for [" + repoName + "]", + new FailPendingRepoTasksTask(repoName, e) + ); } }); } else { @@ -1232,102 +1415,98 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met final StepListener metadataListener = new StepListener<>(); final Repository repo = repositoriesService.repository(snapshot.getRepository()); if (entry.isClone()) { - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute( - ActionRunnable.supply(metadataListener, () -> { - final Metadata existing = repo.getSnapshotGlobalMetadata(entry.source()); - final Metadata.Builder metaBuilder = Metadata.builder(existing); - final Set existingIndices = new HashSet<>(); - for (IndexId index : entry.indices().values()) { - final IndexMetadata indexMetadata = repo.getSnapshotIndexMetaData(repositoryData, entry.source(), index); - existingIndices.add(indexMetadata.getIndex()); - metaBuilder.put(indexMetadata, false); - } - // remove those data streams from metadata for which we are missing indices - Map dataStreamsToCopy = new HashMap<>(); - for (Map.Entry dataStreamEntry : existing.dataStreams().entrySet()) { - if (existingIndices.containsAll(dataStreamEntry.getValue().getIndices())) { - dataStreamsToCopy.put(dataStreamEntry.getKey(), dataStreamEntry.getValue()); - } - } - Map dataStreamAliasesToCopy = - filterDataStreamAliases(dataStreamsToCopy, existing.dataStreamAliases()); - metaBuilder.dataStreams(dataStreamsToCopy, dataStreamAliasesToCopy); - return metaBuilder.build(); - })); + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(metadataListener, () -> { + final Metadata existing = repo.getSnapshotGlobalMetadata(entry.source()); + final Metadata.Builder metaBuilder = Metadata.builder(existing); + final Set existingIndices = new HashSet<>(); + for (IndexId index : entry.indices().values()) { + final IndexMetadata indexMetadata = repo.getSnapshotIndexMetaData(repositoryData, entry.source(), index); + existingIndices.add(indexMetadata.getIndex()); + metaBuilder.put(indexMetadata, false); + } + // remove those data streams from metadata for which we are missing indices + Map dataStreamsToCopy = new HashMap<>(); + for (Map.Entry dataStreamEntry : existing.dataStreams().entrySet()) { + if (existingIndices.containsAll(dataStreamEntry.getValue().getIndices())) { + dataStreamsToCopy.put(dataStreamEntry.getKey(), dataStreamEntry.getValue()); + } + } + Map dataStreamAliasesToCopy = filterDataStreamAliases( + dataStreamsToCopy, + existing.dataStreamAliases() + ); + metaBuilder.dataStreams(dataStreamsToCopy, dataStreamAliasesToCopy); + return metaBuilder.build(); + })); } else { metadataListener.onResponse(metadata); } metadataListener.whenComplete(meta -> { - final Metadata metaForSnapshot = metadataForSnapshot(entry, meta); - - final Map indexSnapshotDetails = new HashMap<>(finalIndices.size()); - for (ObjectObjectCursor shardEntry : entry.shards()) { - indexSnapshotDetails.compute(shardEntry.key.getIndexName(), (indexName, current) -> { - if (current == SnapshotInfo.IndexSnapshotDetails.SKIPPED) { - // already found an unsuccessful shard in this index, skip this shard - return current; - } + final Metadata metaForSnapshot = metadataForSnapshot(entry, meta); + + final Map indexSnapshotDetails = new HashMap<>(finalIndices.size()); + for (ObjectObjectCursor shardEntry : entry.shards()) { + indexSnapshotDetails.compute(shardEntry.key.getIndexName(), (indexName, current) -> { + if (current == SnapshotInfo.IndexSnapshotDetails.SKIPPED) { + // already found an unsuccessful shard in this index, skip this shard + return current; + } - final ShardSnapshotStatus shardSnapshotStatus = shardEntry.value; - if (shardSnapshotStatus.state() != ShardState.SUCCESS) { - // first unsuccessful shard in this index found, record that this index should be skipped - return SnapshotInfo.IndexSnapshotDetails.SKIPPED; - } + final ShardSnapshotStatus shardSnapshotStatus = shardEntry.value; + if (shardSnapshotStatus.state() != ShardState.SUCCESS) { + // first unsuccessful shard in this index found, record that this index should be skipped + return SnapshotInfo.IndexSnapshotDetails.SKIPPED; + } - final ShardSnapshotResult result = shardSnapshotStatus.shardSnapshotResult(); - if (result == null) { - // detailed result not recorded, skip this index - return SnapshotInfo.IndexSnapshotDetails.SKIPPED; - } + final ShardSnapshotResult result = shardSnapshotStatus.shardSnapshotResult(); + if (result == null) { + // detailed result not recorded, skip this index + return SnapshotInfo.IndexSnapshotDetails.SKIPPED; + } - if (current == null) { - return new SnapshotInfo.IndexSnapshotDetails( - 1, - result.getSize(), - result.getSegmentCount() - ); - } else { - return new SnapshotInfo.IndexSnapshotDetails( - current.getShardCount() + 1, - new ByteSizeValue(current.getSize().getBytes() + result.getSize().getBytes()), - Math.max(current.getMaxSegmentsPerShard(), result.getSegmentCount()) - ); - } - }); + if (current == null) { + return new SnapshotInfo.IndexSnapshotDetails(1, result.getSize(), result.getSegmentCount()); + } else { + return new SnapshotInfo.IndexSnapshotDetails( + current.getShardCount() + 1, + new ByteSizeValue(current.getSize().getBytes() + result.getSize().getBytes()), + Math.max(current.getMaxSegmentsPerShard(), result.getSegmentCount()) + ); } - indexSnapshotDetails.entrySet().removeIf(e -> e.getValue().getShardCount() == 0); - - final SnapshotInfo snapshotInfo = new SnapshotInfo( - snapshot.getSnapshotId(), - finalIndices, - entry.dataStreams().stream().filter(metaForSnapshot.dataStreams()::containsKey) - .collect(Collectors.toList()), - entry.partial() ? onlySuccessfulFeatureStates(entry, finalIndices) : entry.featureStates(), - failure, - threadPool.absoluteTimeInMillis(), - entry.partial() ? shardGenerations.totalShards() : entry.shards().size(), - shardFailures, - entry.includeGlobalState(), - // TODO: remove this hack making the metadata mutable once - // https://github.com/elastic/elasticsearch/pull/72776 has been merged - entry.userMetadata() == null ? null : new HashMap<>(entry.userMetadata()), - entry.startTime(), - indexSnapshotDetails); - repo.finalizeSnapshot( - shardGenerations, - repositoryData.getGenId(), - metaForSnapshot, - snapshotInfo, - entry.version(), - state -> stateWithoutSnapshot(state, snapshot), - ActionListener.wrap(newRepoData -> { - completeListenersIgnoringException( - endAndGetListenersToResolve(snapshot), Tuple.tuple(newRepoData, snapshotInfo)); - logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state()); - runNextQueuedOperation(newRepoData, repository, true); - }, e -> handleFinalizationFailure(e, entry, repositoryData))); - }, - e -> handleFinalizationFailure(e, entry, repositoryData)); + }); + } + indexSnapshotDetails.entrySet().removeIf(e -> e.getValue().getShardCount() == 0); + + final SnapshotInfo snapshotInfo = new SnapshotInfo( + snapshot.getSnapshotId(), + finalIndices, + entry.dataStreams().stream().filter(metaForSnapshot.dataStreams()::containsKey).collect(Collectors.toList()), + entry.partial() ? onlySuccessfulFeatureStates(entry, finalIndices) : entry.featureStates(), + failure, + threadPool.absoluteTimeInMillis(), + entry.partial() ? shardGenerations.totalShards() : entry.shards().size(), + shardFailures, + entry.includeGlobalState(), + // TODO: remove this hack making the metadata mutable once + // https://github.com/elastic/elasticsearch/pull/72776 has been merged + entry.userMetadata() == null ? null : new HashMap<>(entry.userMetadata()), + entry.startTime(), + indexSnapshotDetails + ); + repo.finalizeSnapshot( + shardGenerations, + repositoryData.getGenId(), + metaForSnapshot, + snapshotInfo, + entry.version(), + state -> stateWithoutSnapshot(state, snapshot), + ActionListener.wrap(newRepoData -> { + completeListenersIgnoringException(endAndGetListenersToResolve(snapshot), Tuple.tuple(newRepoData, snapshotInfo)); + logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state()); + runNextQueuedOperation(newRepoData, repository, true); + }, e -> handleFinalizationFailure(e, entry, repositoryData)) + ); + }, e -> handleFinalizationFailure(e, entry, repositoryData)); } catch (Exception e) { assert false : new AssertionError(e); handleFinalizationFailure(e, entry, repositoryData); @@ -1353,7 +1532,8 @@ private List onlySuccessfulFeatureStates(SnapshotsInProgres }); // Now remove any feature states which contain any of those indices, as the feature state is not intact and not safely restorable - return entry.featureStates().stream() + return entry.featureStates() + .stream() .filter(stateInfo -> finalIndices.containsAll(stateInfo.getIndices())) .filter(stateInfo -> stateInfo.getIndices().stream().anyMatch(indicesWithUnsuccessfulShards::contains) == false) .collect(Collectors.toList()); @@ -1386,10 +1566,11 @@ private void handleFinalizationFailure(Exception e, SnapshotsInProgress.Entry en if (ExceptionsHelper.unwrap(e, NotMasterException.class, FailedToCommitClusterStateException.class) != null) { // Failure due to not being master any more, don't try to remove snapshot from cluster state the next master // will try ending this snapshot again - logger.debug(() -> new ParameterizedMessage( - "[{}] failed to update cluster state during snapshot finalization", snapshot), e); - failSnapshotCompletionListeners(snapshot, - new SnapshotException(snapshot, "Failed to update cluster state during snapshot finalization", e)); + logger.debug(() -> new ParameterizedMessage("[{}] failed to update cluster state during snapshot finalization", snapshot), e); + failSnapshotCompletionListeners( + snapshot, + new SnapshotException(snapshot, "Failed to update cluster state during snapshot finalization", e) + ); failAllListenersOnMasterFailOver(e); } else { logger.warn(() -> new ParameterizedMessage("[{}] failed to finalize snapshot", snapshot), e); @@ -1432,10 +1613,12 @@ private void runReadyDeletions(RepositoryData repositoryData, String repository) @Override public ClusterState execute(ClusterState currentState) { - assert readyDeletions(currentState).v1() == currentState : - "Deletes should have been set to ready by finished snapshot deletes and finalizations"; - for (SnapshotDeletionsInProgress.Entry entry : - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { + assert readyDeletions(currentState).v1() == currentState + : "Deletes should have been set to ready by finished snapshot deletes and finalizations"; + for (SnapshotDeletionsInProgress.Entry entry : currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ).getEntries()) { if (entry.repository().equals(repository) && entry.state() == SnapshotDeletionsInProgress.State.STARTED) { deletionToRun = entry; break; @@ -1472,8 +1655,10 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * @return tuple of an updated cluster state and currently executable snapshot delete operations */ private static Tuple> readyDeletions(ClusterState currentState) { - final SnapshotDeletionsInProgress deletions = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final SnapshotDeletionsInProgress deletions = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); if (deletions.hasDeletionsInProgress() == false) { return Tuple.tuple(currentState, List.of()); } @@ -1485,9 +1670,12 @@ private static Tuple> read final List newDeletes = new ArrayList<>(); for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) { final String repo = entry.repository(); - if (repositoriesSeen.add(entry.repository()) && entry.state() == SnapshotDeletionsInProgress.State.WAITING - && snapshotsInProgress.entries().stream() - .filter(se -> se.repository().equals(repo)).noneMatch(SnapshotsService::isWritingToRepository)) { + if (repositoriesSeen.add(entry.repository()) + && entry.state() == SnapshotDeletionsInProgress.State.WAITING + && snapshotsInProgress.entries() + .stream() + .filter(se -> se.repository().equals(repo)) + .noneMatch(SnapshotsService::isWritingToRepository)) { changed = true; final SnapshotDeletionsInProgress.Entry newEntry = entry.started(); readyDeletions.add(newEntry); @@ -1496,8 +1684,14 @@ private static Tuple> read newDeletes.add(entry); } } - return Tuple.tuple(changed ? ClusterState.builder(currentState).putCustom( - SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.of(newDeletes)).build() : currentState, readyDeletions); + return Tuple.tuple( + changed + ? ClusterState.builder(currentState) + .putCustom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.of(newDeletes)) + .build() + : currentState, + readyDeletions + ); } /** @@ -1543,16 +1737,24 @@ private void removeFailedSnapshotFromClusterState(Snapshot snapshot, Exception f public ClusterState execute(ClusterState currentState) { final ClusterState updatedState = stateWithoutSnapshot(currentState, snapshot); // now check if there are any delete operations that refer to the just failed snapshot and remove the snapshot from them - return updateWithSnapshots(updatedState, null, deletionsWithoutSnapshots( + return updateWithSnapshots( + updatedState, + null, + deletionsWithoutSnapshots( updatedState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY), - Collections.singletonList(snapshot.getSnapshotId()), snapshot.getRepository())); + Collections.singletonList(snapshot.getSnapshotId()), + snapshot.getRepository() + ) + ); } @Override public void onFailure(String source, Exception e) { logger.warn(() -> new ParameterizedMessage("[{}] failed to remove snapshot metadata", snapshot), e); failSnapshotCompletionListeners( - snapshot, new SnapshotException(snapshot, "Failed to remove snapshot from cluster state", e)); + snapshot, + new SnapshotException(snapshot, "Failed to remove snapshot from cluster state", e) + ); failAllListenersOnMasterFailOver(e); } @@ -1583,8 +1785,11 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * @return updated {@link SnapshotDeletionsInProgress} or {@code null} if unchanged */ @Nullable - private static SnapshotDeletionsInProgress deletionsWithoutSnapshots(SnapshotDeletionsInProgress deletions, - Collection snapshotIds, String repository) { + private static SnapshotDeletionsInProgress deletionsWithoutSnapshots( + SnapshotDeletionsInProgress deletions, + Collection snapshotIds, + String repository + ) { boolean changed = false; List updatedEntries = new ArrayList<>(deletions.getEntries().size()); for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) { @@ -1618,8 +1823,13 @@ public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionLis final String[] snapshotNames = request.snapshots(); final String repoName = request.repository(); - logger.info(() -> new ParameterizedMessage("deleting snapshots [{}] from repository [{}]", - Strings.arrayToCommaDelimitedString(snapshotNames), repoName)); + logger.info( + () -> new ParameterizedMessage( + "deleting snapshots [{}] from repository [{}]", + Strings.arrayToCommaDelimitedString(snapshotNames), + repoName + ) + ); final Repository repository = repositoriesService.repository(repoName); repository.executeConsistentStateUpdate(repositoryData -> new ClusterStateUpdateTask(request.masterNodeTimeout()) { @@ -1641,29 +1851,40 @@ public ClusterState execute(ClusterState currentState) { final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List snapshotEntries = findInProgressSnapshots(snapshots, snapshotNames, repoName); final List snapshotIds = matchingSnapshotIds( - snapshotEntries.stream().map(e -> e.snapshot().getSnapshotId()).collect(Collectors.toList()), repositoryData, - snapshotNames, repoName); + snapshotEntries.stream().map(e -> e.snapshot().getSnapshotId()).collect(Collectors.toList()), + repositoryData, + snapshotNames, + repoName + ); if (snapshotIds.isEmpty()) { return currentState; } final Set activeCloneSources = snapshots.entries() - .stream() - .filter(SnapshotsInProgress.Entry::isClone) - .map(SnapshotsInProgress.Entry::source) - .collect(Collectors.toSet()); + .stream() + .filter(SnapshotsInProgress.Entry::isClone) + .map(SnapshotsInProgress.Entry::source) + .collect(Collectors.toSet()); for (SnapshotId snapshotId : snapshotIds) { if (activeCloneSources.contains(snapshotId)) { - throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotId), - "cannot delete snapshot while it is being cloned"); + throw new ConcurrentSnapshotExecutionException( + new Snapshot(repoName, snapshotId), + "cannot delete snapshot while it is being cloned" + ); } } - final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - final RepositoryCleanupInProgress repositoryCleanupInProgress = - currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); + final RepositoryCleanupInProgress repositoryCleanupInProgress = currentState.custom( + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress.EMPTY + ); if (repositoryCleanupInProgress.hasCleanupInProgress()) { - throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), - "cannot delete snapshots while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]"); + throw new ConcurrentSnapshotExecutionException( + new Snapshot(repoName, snapshotIds.get(0)), + "cannot delete snapshots while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]" + ); } final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); // don't allow snapshot deletions while a restore is taking place, @@ -1672,67 +1893,80 @@ public ClusterState execute(ClusterState currentState) { for (RestoreInProgress.Entry entry : restoreInProgress) { if (repoName.equals(entry.snapshot().getRepository()) && snapshotIds.contains(entry.snapshot().getSnapshotId())) { - throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), - "cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]"); + throw new ConcurrentSnapshotExecutionException( + new Snapshot(repoName, snapshotIds.get(0)), + "cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]" + ); } } // Snapshot ids that will have to be physically deleted from the repository final Set snapshotIdsRequiringCleanup = new HashSet<>(snapshotIds); - final SnapshotsInProgress updatedSnapshots = SnapshotsInProgress.of(snapshots.entries().stream() - .map(existing -> { - if (existing.state() == State.STARTED && - snapshotIdsRequiringCleanup.contains(existing.snapshot().getSnapshotId())) { - // snapshot is started - mark every non completed shard as aborted - final SnapshotsInProgress.Entry abortedEntry = existing.abort(); - if (abortedEntry == null) { - // No work has been done for this snapshot yet so we remove it from the cluster state directly - final Snapshot existingNotYetStartedSnapshot = existing.snapshot(); - // Adding the snapshot to #endingSnapshots since we still have to resolve its listeners to not trip - // any leaked listener assertions - if (endingSnapshots.add(existingNotYetStartedSnapshot)) { - completedNoCleanup.add(existingNotYetStartedSnapshot); - } - snapshotIdsRequiringCleanup.remove(existingNotYetStartedSnapshot.getSnapshotId()); - } else if (abortedEntry.state().completed()) { - completedWithCleanup.add(abortedEntry); - } - return abortedEntry; + final SnapshotsInProgress updatedSnapshots = SnapshotsInProgress.of(snapshots.entries().stream().map(existing -> { + if (existing.state() == State.STARTED && snapshotIdsRequiringCleanup.contains(existing.snapshot().getSnapshotId())) { + // snapshot is started - mark every non completed shard as aborted + final SnapshotsInProgress.Entry abortedEntry = existing.abort(); + if (abortedEntry == null) { + // No work has been done for this snapshot yet so we remove it from the cluster state directly + final Snapshot existingNotYetStartedSnapshot = existing.snapshot(); + // Adding the snapshot to #endingSnapshots since we still have to resolve its listeners to not trip + // any leaked listener assertions + if (endingSnapshots.add(existingNotYetStartedSnapshot)) { + completedNoCleanup.add(existingNotYetStartedSnapshot); } - return existing; - }).filter(Objects::nonNull).collect(Collectors.toUnmodifiableList())); + snapshotIdsRequiringCleanup.remove(existingNotYetStartedSnapshot.getSnapshotId()); + } else if (abortedEntry.state().completed()) { + completedWithCleanup.add(abortedEntry); + } + return abortedEntry; + } + return existing; + }).filter(Objects::nonNull).collect(Collectors.toUnmodifiableList())); if (snapshotIdsRequiringCleanup.isEmpty()) { // We only saw snapshots that could be removed from the cluster state right away, no need to update the deletions return updateWithSnapshots(currentState, updatedSnapshots, null); } // add the snapshot deletion to the cluster state - final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries().stream().filter(entry -> - entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.WAITING) - .findFirst().orElse(null); + final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries() + .stream() + .filter(entry -> entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.WAITING) + .findFirst() + .orElse(null); if (replacedEntry == null) { - final Optional foundDuplicate = - deletionsInProgress.getEntries().stream().filter(entry -> - entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.STARTED - && entry.getSnapshots().containsAll(snapshotIds)).findFirst(); + final Optional foundDuplicate = deletionsInProgress.getEntries() + .stream() + .filter( + entry -> entry.repository().equals(repoName) + && entry.state() == SnapshotDeletionsInProgress.State.STARTED + && entry.getSnapshots().containsAll(snapshotIds) + ) + .findFirst(); if (foundDuplicate.isPresent()) { newDelete = foundDuplicate.get(); reusedExistingDelete = true; return currentState; } newDelete = new SnapshotDeletionsInProgress.Entry( - List.copyOf(snapshotIdsRequiringCleanup), - repoName, - threadPool.absoluteTimeInMillis(), - repositoryData.getGenId(), - updatedSnapshots.entries().stream().filter(entry -> repoName.equals(entry.repository())).noneMatch( - SnapshotsService::isWritingToRepository) - && deletionsInProgress.hasExecutingDeletion(repoName) == false - ? SnapshotDeletionsInProgress.State.STARTED : SnapshotDeletionsInProgress.State.WAITING); + List.copyOf(snapshotIdsRequiringCleanup), + repoName, + threadPool.absoluteTimeInMillis(), + repositoryData.getGenId(), + updatedSnapshots.entries() + .stream() + .filter(entry -> repoName.equals(entry.repository())) + .noneMatch(SnapshotsService::isWritingToRepository) + && deletionsInProgress.hasExecutingDeletion(repoName) == false + ? SnapshotDeletionsInProgress.State.STARTED + : SnapshotDeletionsInProgress.State.WAITING + ); } else { newDelete = replacedEntry.withAddedSnapshots(snapshotIdsRequiringCleanup); } - return updateWithSnapshots(currentState, updatedSnapshots, - (replacedEntry == null ? deletionsInProgress : deletionsInProgress.withRemovedEntry(replacedEntry.uuid())) - .withAddedEntry(newDelete)); + return updateWithSnapshots( + currentState, + updatedSnapshots, + (replacedEntry == null ? deletionsInProgress : deletionsInProgress.withRemovedEntry(replacedEntry.uuid())) + .withAddedEntry(newDelete) + ); } @Override @@ -1747,8 +1981,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS logger.info("snapshots {} aborted", completedNoCleanup); } for (Snapshot snapshot : completedNoCleanup) { - failSnapshotCompletionListeners(snapshot, - new SnapshotException(snapshot, SnapshotsInProgress.ABORTED_FAILURE_TEXT)); + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, SnapshotsInProgress.ABORTED_FAILURE_TEXT)); } if (newDelete == null) { listener.onResponse(null); @@ -1773,10 +2006,15 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS }, "delete snapshot", listener::onFailure); } - private static List matchingSnapshotIds(List inProgress, RepositoryData repositoryData, - String[] snapshotsOrPatterns, String repositoryName) { - final Map allSnapshotIds = repositoryData.getSnapshotIds().stream().collect( - Collectors.toMap(SnapshotId::getName, Function.identity())); + private static List matchingSnapshotIds( + List inProgress, + RepositoryData repositoryData, + String[] snapshotsOrPatterns, + String repositoryName + ) { + final Map allSnapshotIds = repositoryData.getSnapshotIds() + .stream() + .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); final Set foundSnapshots = new HashSet<>(inProgress); for (String snapshotOrPattern : snapshotsOrPatterns) { if (Regex.isSimpleMatchPattern(snapshotOrPattern)) { @@ -1800,12 +2038,14 @@ private static List matchingSnapshotIds(List inProgress, } // Return in-progress snapshot entries by name and repository in the given cluster state or null if none is found - private static List findInProgressSnapshots(SnapshotsInProgress snapshots, String[] snapshotNames, - String repositoryName) { + private static List findInProgressSnapshots( + SnapshotsInProgress snapshots, + String[] snapshotNames, + String repositoryName + ) { List entries = new ArrayList<>(); for (SnapshotsInProgress.Entry entry : snapshots.entries()) { - if (entry.repository().equals(repositoryName) - && Regex.simpleMatch(snapshotNames, entry.snapshot().getSnapshotId().getName())) { + if (entry.repository().equals(repositoryName) && Regex.simpleMatch(snapshotNames, entry.snapshot().getSnapshotId().getName())) { entries.add(entry); } } @@ -1850,14 +2090,18 @@ private void addDeleteListener(String deleteUUID, ActionListener listener) public Version minCompatibleVersion(Version minNodeVersion, RepositoryData repositoryData, @Nullable Collection excluded) { Version minCompatVersion = minNodeVersion; final Collection snapshotIds = repositoryData.getSnapshotIds(); - for (SnapshotId snapshotId : snapshotIds.stream().filter(excluded == null ? sn -> true : Predicate.not(excluded::contains)) - .collect(Collectors.toList())) { + for (SnapshotId snapshotId : snapshotIds.stream() + .filter(excluded == null ? sn -> true : Predicate.not(excluded::contains)) + .collect(Collectors.toList())) { final Version known = repositoryData.getVersion(snapshotId); // If we don't have the version cached in the repository data yet we load it from the snapshot info blobs if (known == null) { - assert repositoryData.shardGenerations().totalShards() == 0 : - "Saw shard generations [" + repositoryData.shardGenerations() + - "] but did not have versions tracked for snapshot [" + snapshotId + "]"; + assert repositoryData.shardGenerations().totalShards() == 0 + : "Saw shard generations [" + + repositoryData.shardGenerations() + + "] but did not have versions tracked for snapshot [" + + snapshotId + + "]"; return OLD_SNAPSHOT_FORMAT; } else { minCompatVersion = minCompatVersion.before(known) ? minCompatVersion : known; @@ -1906,16 +2150,21 @@ private void deleteSnapshotsFromRepository(SnapshotDeletionsInProgress.Entry del repositoriesService.getRepositoryData(deleteEntry.repository(), new ActionListener<>() { @Override public void onResponse(RepositoryData repositoryData) { - assert repositoryData.getGenId() == expectedRepoGen : - "Repository generation should not change as long as a ready delete is found in the cluster state but found [" - + expectedRepoGen + "] in cluster state and [" + repositoryData.getGenId() + "] in the repository"; + assert repositoryData.getGenId() == expectedRepoGen + : "Repository generation should not change as long as a ready delete is found in the cluster state but found [" + + expectedRepoGen + + "] in cluster state and [" + + repositoryData.getGenId() + + "] in the repository"; deleteSnapshotsFromRepository(deleteEntry, repositoryData, minNodeVersion); } @Override public void onFailure(Exception e) { - clusterService.submitStateUpdateTask("fail repo tasks for [" + deleteEntry.repository() + "]", - new FailPendingRepoTasksTask(deleteEntry.repository(), e)); + clusterService.submitStateUpdateTask( + "fail repo tasks for [" + deleteEntry.repository() + "]", + new FailPendingRepoTasksTask(deleteEntry.repository(), e) + ); } }); } @@ -1926,22 +2175,25 @@ public void onFailure(Exception e) { * @param repositoryData the {@link RepositoryData} of the repository to delete from * @param minNodeVersion minimum node version in the cluster */ - private void deleteSnapshotsFromRepository(SnapshotDeletionsInProgress.Entry deleteEntry, - RepositoryData repositoryData, Version minNodeVersion) { + private void deleteSnapshotsFromRepository( + SnapshotDeletionsInProgress.Entry deleteEntry, + RepositoryData repositoryData, + Version minNodeVersion + ) { if (repositoryOperations.startDeletion(deleteEntry.uuid())) { assert currentlyFinalizing.contains(deleteEntry.repository()); final List snapshotIds = deleteEntry.getSnapshots(); - assert deleteEntry.state() == SnapshotDeletionsInProgress.State.STARTED : - "incorrect state for entry [" + deleteEntry + "]"; - repositoriesService.repository(deleteEntry.repository()).deleteSnapshots( - snapshotIds, - repositoryData.getGenId(), - minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), - ActionListener.wrap(updatedRepoData -> { + assert deleteEntry.state() == SnapshotDeletionsInProgress.State.STARTED : "incorrect state for entry [" + deleteEntry + "]"; + repositoriesService.repository(deleteEntry.repository()) + .deleteSnapshots( + snapshotIds, + repositoryData.getGenId(), + minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), + ActionListener.wrap(updatedRepoData -> { logger.info("snapshots {} deleted", snapshotIds); removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); - }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) - )); + }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData)) + ); } } @@ -1954,8 +2206,11 @@ private void deleteSnapshotsFromRepository(SnapshotDeletionsInProgress.Entry del * successfully * @param repositoryData current {@link RepositoryData} for the repository we just ran the delete on. */ - private void removeSnapshotDeletionFromClusterState(final SnapshotDeletionsInProgress.Entry deleteEntry, - @Nullable final Exception failure, final RepositoryData repositoryData) { + private void removeSnapshotDeletionFromClusterState( + final SnapshotDeletionsInProgress.Entry deleteEntry, + @Nullable final Exception failure, + final RepositoryData repositoryData + ) { final ClusterStateUpdateTask clusterStateUpdateTask; if (failure == null) { // If we didn't have a failure during the snapshot delete we will remove all snapshot ids that the delete successfully removed @@ -1964,16 +2219,22 @@ private void removeSnapshotDeletionFromClusterState(final SnapshotDeletionsInPro clusterStateUpdateTask = new RemoveSnapshotDeletionAndContinueTask(deleteEntry, repositoryData) { @Override protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgress deletions) { - final SnapshotDeletionsInProgress updatedDeletions = - deletionsWithoutSnapshots(deletions, deleteEntry.getSnapshots(), deleteEntry.repository()); + final SnapshotDeletionsInProgress updatedDeletions = deletionsWithoutSnapshots( + deletions, + deleteEntry.getSnapshots(), + deleteEntry.repository() + ); return updatedDeletions == null ? deletions : updatedDeletions; } @Override protected void handleListeners(List> deleteListeners) { assert repositoryData.getSnapshotIds().stream().noneMatch(deleteEntry.getSnapshots()::contains) - : "Repository data contained snapshot ids " + repositoryData.getSnapshotIds() - + " that should should been deleted by [" + deleteEntry + "]"; + : "Repository data contained snapshot ids " + + repositoryData.getSnapshotIds() + + " that should should been deleted by [" + + deleteEntry + + "]"; completeListenersIgnoringException(deleteListeners, null); } }; @@ -2005,19 +2266,16 @@ private void failAllListenersOnMasterFailOver(Exception e) { for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); } - final Exception wrapped = - new RepositoryException("_all", "Failed to update cluster state during repository operation", e); - for (Iterator>> iterator = snapshotDeletionListeners.values().iterator(); - iterator.hasNext(); ) { + final Exception wrapped = new RepositoryException("_all", "Failed to update cluster state during repository operation", e); + for (Iterator>> iterator = snapshotDeletionListeners.values().iterator(); iterator.hasNext();) { final List> listeners = iterator.next(); iterator.remove(); failListenersIgnoringException(listeners, wrapped); } - assert snapshotDeletionListeners.isEmpty() : - "No new listeners should have been added but saw " + snapshotDeletionListeners; + assert snapshotDeletionListeners.isEmpty() : "No new listeners should have been added but saw " + snapshotDeletionListeners; } else { - assert false : - new AssertionError("Modifying snapshot state should only ever fail because we failed to publish new state", e); + assert false + : new AssertionError("Modifying snapshot state should only ever fail because we failed to publish new state", e); logger.error("Unexpected failure during cluster state update", e); } currentlyFinalizing.clear(); @@ -2055,7 +2313,8 @@ public ClusterState execute(ClusterState currentState) { } final SnapshotDeletionsInProgress newDeletions = filterDeletions(updatedDeletions); final Tuple> res = readyDeletions( - updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions)); + updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions) + ); readyDeletions = res.v2(); return res.v1(); } @@ -2088,13 +2347,13 @@ public final void clusterStateProcessed(String source, ClusterState oldState, Cl } else { leaveRepoLoop(deleteEntry.repository()); assert readyDeletions.stream().noneMatch(entry -> entry.repository().equals(deleteEntry.repository())) - : "New finalizations " + newFinalizations + " added even though deletes " + readyDeletions + " are ready"; + : "New finalizations " + newFinalizations + " added even though deletes " + readyDeletions + " are ready"; for (SnapshotsInProgress.Entry entry : newFinalizations) { endSnapshot(entry, newState.metadata(), repositoryData); } } // TODO: be more efficient here, we could collect newly ready shard clones as we compute them and then directly start them - // instead of looping over all possible clones to execute + // instead of looping over all possible clones to execute startExecutableClones(newState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), null); } @@ -2124,8 +2383,7 @@ public final void clusterStateProcessed(String source, ClusterState oldState, Cl * @return updated snapshot in progress instance or {@code null} if there are no changes to it */ @Nullable - private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState, - SnapshotDeletionsInProgress updatedDeletions) { + private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState, SnapshotDeletionsInProgress updatedDeletions) { final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List snapshotEntries = new ArrayList<>(); @@ -2144,18 +2402,18 @@ private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState if (entry.repository().equals(repoName)) { if (entry.state().completed() == false) { // TODO: dry up redundant computation and code between clone and non-clone case, in particular reuse - // `inFlightShardStates` across both clone and standard snapshot code + // `inFlightShardStates` across both clone and standard snapshot code if (entry.isClone()) { // Collect waiting shards from that entry that we can assign now that we are done with the deletion final List canBeUpdated = new ArrayList<>(); for (ObjectObjectCursor value : entry.clones()) { if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED) - && alreadyReassigned(value.key.indexName(), value.key.shardId(), reassignedShardIds) == false) { + && alreadyReassigned(value.key.indexName(), value.key.shardId(), reassignedShardIds) == false) { canBeUpdated.add(value.key); } } // TODO: the below logic is very similar to that in #startCloning and both could be dried up against each other - // also the code for standard snapshots could make use of this breakout as well + // also the code for standard snapshots could make use of this breakout as well if (canBeUpdated.isEmpty() || updatedDeletions.hasExecutingDeletion(repoName)) { // No shards can be updated in this snapshot so we just add it as is again snapshotEntries.add(entry); @@ -2164,14 +2422,21 @@ && alreadyReassigned(value.key.indexName(), value.key.shardId(), reassignedShard inFlightShardStates = InFlightShardSnapshotStates.forRepo(repoName, snapshotsInProgress.entries()); } final ImmutableOpenMap.Builder updatedAssignmentsBuilder = - ImmutableOpenMap.builder(entry.clones()); + ImmutableOpenMap.builder(entry.clones()); for (RepositoryShardId shardId : canBeUpdated) { if (inFlightShardStates.isActive(shardId.indexName(), shardId.shardId()) == false) { markShardReassigned(shardId.indexName(), shardId.shardId(), reassignedShardIds); - updatedAssignmentsBuilder.put(shardId, - new ShardSnapshotStatus(localNodeId, - inFlightShardStates.generationForShard( - shardId.index(), shardId.shardId(), repositoryData.shardGenerations()))); + updatedAssignmentsBuilder.put( + shardId, + new ShardSnapshotStatus( + localNodeId, + inFlightShardStates.generationForShard( + shardId.index(), + shardId.shardId(), + repositoryData.shardGenerations() + ) + ) + ); } } snapshotEntries.add(entry.withClones(updatedAssignmentsBuilder.build())); @@ -2182,7 +2447,7 @@ && alreadyReassigned(value.key.indexName(), value.key.shardId(), reassignedShard final List canBeUpdated = new ArrayList<>(); for (ObjectObjectCursor value : entry.shards()) { if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED) - && alreadyReassigned(value.key.getIndexName(), value.key.getId(), reassignedShardIds) == false) { + && alreadyReassigned(value.key.getIndexName(), value.key.getId(), reassignedShardIds) == false) { canBeUpdated.add(value.key); } } @@ -2191,18 +2456,25 @@ && alreadyReassigned(value.key.getIndexName(), value.key.getId(), reassignedShar snapshotEntries.add(entry); } else { if (shardAssignments == null) { - shardAssignments = shards(snapshotsInProgress, - updatedDeletions, currentState.metadata(), currentState.routingTable(), entry.indices().values(), - entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION), repositoryData, repoName); + shardAssignments = shards( + snapshotsInProgress, + updatedDeletions, + currentState.metadata(), + currentState.routingTable(), + entry.indices().values(), + entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION), + repositoryData, + repoName + ); } - final ImmutableOpenMap.Builder updatedAssignmentsBuilder = - ImmutableOpenMap.builder(entry.shards()); + final ImmutableOpenMap.Builder updatedAssignmentsBuilder = ImmutableOpenMap + .builder(entry.shards()); for (ShardId shardId : canBeUpdated) { final ShardSnapshotStatus updated = shardAssignments.get(shardId); if (updated == null) { // We don't have a new assignment for this shard because its index was concurrently deleted - assert currentState.routingTable().hasIndex(shardId.getIndex()) == false : - "Missing assignment for [" + shardId + "]"; + assert currentState.routingTable().hasIndex(shardId.getIndex()) == false + : "Missing assignment for [" + shardId + "]"; updatedAssignmentsBuilder.put(shardId, ShardSnapshotStatus.MISSING); } else { markShardReassigned(shardId.getIndexName(), shardId.id(), reassignedShardIds); @@ -2250,9 +2522,11 @@ private boolean alreadyReassigned(String indexName, int shardId, Map void completeListenersIgnoringException(@Nullable List shards( - SnapshotsInProgress snapshotsInProgress, SnapshotDeletionsInProgress deletionsInProgress, - Metadata metadata, RoutingTable routingTable, Collection indices, boolean useShardGenerations, - RepositoryData repositoryData, String repoName) { + SnapshotsInProgress snapshotsInProgress, + SnapshotDeletionsInProgress deletionsInProgress, + Metadata metadata, + RoutingTable routingTable, + Collection indices, + boolean useShardGenerations, + RepositoryData repositoryData, + String repoName + ) { ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); final ShardGenerations shardGenerations = repositoryData.shardGenerations(); - final InFlightShardSnapshotStates inFlightShardStates = - InFlightShardSnapshotStates.forRepo(repoName, snapshotsInProgress.entries()); + final InFlightShardSnapshotStates inFlightShardStates = InFlightShardSnapshotStates.forRepo( + repoName, + snapshotsInProgress.entries() + ); final boolean readyToExecute = deletionsInProgress.hasExecutingDeletion(repoName) == false; for (IndexId index : indices) { final String indexName = index.getName(); @@ -2358,8 +2640,12 @@ private static ShardSnapshotStatus initShardSnapshotStatus(String shardRepoGener } else if (primary.relocating() || primary.initializing()) { shardSnapshotStatus = new ShardSnapshotStatus(primary.currentNodeId(), ShardState.WAITING, shardRepoGeneration); } else if (primary.started() == false) { - shardSnapshotStatus = new ShardSnapshotStatus(primary.currentNodeId(), ShardState.MISSING, - "primary shard hasn't been started yet", shardRepoGeneration); + shardSnapshotStatus = new ShardSnapshotStatus( + primary.currentNodeId(), + ShardState.MISSING, + "primary shard hasn't been started yet", + shardRepoGeneration + ); } else { shardSnapshotStatus = new ShardSnapshotStatus(primary.currentNodeId(), shardRepoGeneration); } @@ -2370,15 +2656,15 @@ private static ShardSnapshotStatus initShardSnapshotStatus(String shardRepoGener * Returns the data streams that are currently being snapshotted (with partial == false) and that are contained in the * indices-to-check set. */ - public static Set snapshottingDataStreams(final ClusterState currentState, - final Set dataStreamsToCheck) { + public static Set snapshottingDataStreams(final ClusterState currentState, final Set dataStreamsToCheck) { final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); if (snapshots == null) { return emptySet(); } Map dataStreams = currentState.metadata().dataStreams(); - return snapshots.entries().stream() + return snapshots.entries() + .stream() .filter(e -> e.partial() == false) .flatMap(e -> e.dataStreams().stream()) .filter(ds -> dataStreams.containsKey(ds) && dataStreamsToCheck.contains(ds)) @@ -2420,10 +2706,13 @@ public static Set snapshottingIndices(final ClusterState currentState, fi * that don't exist in the provided data streams. * @return The filtered data streams aliases only referring to data streams in the provided data streams. */ - static Map filterDataStreamAliases(Map dataStreams, - Map dataStreamAliases) { + static Map filterDataStreamAliases( + Map dataStreams, + Map dataStreamAliases + ) { - return dataStreamAliases.values().stream() + return dataStreamAliases.values() + .stream() .filter(alias -> alias.getDataStreams().stream().anyMatch(dataStreams::containsKey)) .map(alias -> alias.intersect(dataStreams::containsKey)) .collect(Collectors.toMap(DataStreamAlias::getName, Function.identity())); @@ -2460,16 +2749,13 @@ protected void doClose() { */ public boolean assertAllListenersResolved() { final DiscoveryNode localNode = clusterService.localNode(); - assert endingSnapshots.isEmpty() : "Found leaked ending snapshots " + endingSnapshots - + " on [" + localNode + "]"; - assert snapshotCompletionListeners.isEmpty() : "Found leaked snapshot completion listeners " + snapshotCompletionListeners - + " on [" + localNode + "]"; - assert currentlyFinalizing.isEmpty() : "Found leaked finalizations " + currentlyFinalizing - + " on [" + localNode + "]"; - assert snapshotDeletionListeners.isEmpty() : "Found leaked snapshot delete listeners " + snapshotDeletionListeners - + " on [" + localNode + "]"; - assert repositoryOperations.isEmpty() : "Found leaked snapshots to finalize " + repositoryOperations - + " on [" + localNode + "]"; + assert endingSnapshots.isEmpty() : "Found leaked ending snapshots " + endingSnapshots + " on [" + localNode + "]"; + assert snapshotCompletionListeners.isEmpty() + : "Found leaked snapshot completion listeners " + snapshotCompletionListeners + " on [" + localNode + "]"; + assert currentlyFinalizing.isEmpty() : "Found leaked finalizations " + currentlyFinalizing + " on [" + localNode + "]"; + assert snapshotDeletionListeners.isEmpty() + : "Found leaked snapshot delete listeners " + snapshotDeletionListeners + " on [" + localNode + "]"; + assert repositoryOperations.isEmpty() : "Found leaked snapshots to finalize " + repositoryOperations + " on [" + localNode + "]"; return true; } @@ -2492,9 +2778,11 @@ public boolean assertAllListenersResolved() { * * Package private to allow for tests. */ - static final ClusterStateTaskExecutor SHARD_STATE_EXECUTOR = (currentState, tasks) -> - ClusterStateTaskExecutor.ClusterTasksResult.builder().successes(tasks).build( - new SnapshotShardsUpdateContext(currentState, tasks).computeUpdatedState()); + static final ClusterStateTaskExecutor SHARD_STATE_EXECUTOR = ( + currentState, + tasks) -> ClusterStateTaskExecutor.ClusterTasksResult.builder() + .successes(tasks) + .build(new SnapshotShardsUpdateContext(currentState, tasks).computeUpdatedState()); private static boolean isQueued(@Nullable ShardSnapshotStatus status) { return status != null && status.state() == ShardState.QUEUED; @@ -2527,16 +2815,19 @@ private static final class SnapshotShardsUpdateContext { } ClusterState computeUpdatedState() { - final List oldEntries - = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries(); + final List oldEntries = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .entries(); final List newEntries = new ArrayList<>(oldEntries.size()); for (SnapshotsInProgress.Entry entry : oldEntries) { newEntries.add(applyToEntry(entry)); } if (changedCount > 0) { - logger.trace("changed cluster state triggered by [{}] snapshot state updates and resulted in starting " + - "[{}] shard snapshots", changedCount, startedCount); + logger.trace( + "changed cluster state triggered by [{}] snapshot state updates and resulted in starting " + "[{}] shard snapshots", + changedCount, + startedCount + ); return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(newEntries)).build(); } return currentState; @@ -2599,8 +2890,11 @@ SnapshotsInProgress.Entry computeUpdatedEntry() { } if (shardsBuilder != null) { - assert clonesBuilder == null : "Should not have updated clones when updating shard snapshots but saw " + clonesBuilder - + " as well as " + shardsBuilder; + assert clonesBuilder == null + : "Should not have updated clones when updating shard snapshots but saw " + + clonesBuilder + + " as well as " + + shardsBuilder; return entry.withShardStates(shardsBuilder.build()); } else if (clonesBuilder != null) { return entry.withClones(clonesBuilder.build()); @@ -2618,8 +2912,12 @@ SnapshotsInProgress.Entry computeUpdatedEntry() { * @param shardId shard identifier of shard to start operation for * @param either {@link ShardId} for snapshots or {@link RepositoryShardId} for clones */ - private void startShardOperation(ImmutableOpenMap.Builder newStates, String nodeId, - String generation, T shardId) { + private void startShardOperation( + ImmutableOpenMap.Builder newStates, + String nodeId, + String generation, + T shardId + ) { startShardOperation(newStates, shardId, new ShardSnapshotStatus(nodeId, generation)); } @@ -2631,18 +2929,29 @@ private void startShardOperation(ImmutableOpenMap.Builder either {@link ShardId} for snapshots or {@link RepositoryShardId} for clones */ - private void startShardOperation(ImmutableOpenMap.Builder newStates, T shardId, - ShardSnapshotStatus newState) { - logger.trace("[{}] Starting [{}] on [{}] with generation [{}]", entry.snapshot(), shardId, newState.nodeId(), - newState.generation()); + private void startShardOperation( + ImmutableOpenMap.Builder newStates, + T shardId, + ShardSnapshotStatus newState + ) { + logger.trace( + "[{}] Starting [{}] on [{}] with generation [{}]", + entry.snapshot(), + shardId, + newState.nodeId(), + newState.generation() + ); newStates.put(shardId, newState); iterator.remove(); startedCount++; } - private void executeShardSnapshotUpdate(ImmutableOpenMap existingStates, - Supplier> newStates, - ShardSnapshotUpdate updateSnapshotState, T updatedShard) { + private void executeShardSnapshotUpdate( + ImmutableOpenMap existingStates, + Supplier> newStates, + ShardSnapshotUpdate updateSnapshotState, + T updatedShard + ) { assert updateSnapshotState.snapshot.equals(entry.snapshot()); final ShardSnapshotStatus existing = existingStates.get(updatedShard); if (existing == null) { @@ -2657,8 +2966,12 @@ private void executeShardSnapshotUpdate(ImmutableOpenMap clone : entry.clones()) { if (clone.value.state() == ShardState.INIT) { - runReadyClone(entry.snapshot(), entry.source(), clone.value, clone.key, - repositoriesService.repository(entry.repository())); + runReadyClone( + entry.snapshot(), + entry.source(), + clone.value, + clone.key, + repositoriesService.repository(entry.repository()) + ); } } } } } - private class UpdateSnapshotStatusAction - extends TransportMasterNodeAction { - UpdateSnapshotStatusAction(TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { - super(UPDATE_SNAPSHOT_STATUS_ACTION_NAME, false, transportService, clusterService, threadPool, - actionFilters, UpdateIndexShardSnapshotStatusRequest::new, indexNameExpressionResolver, - in -> ActionResponse.Empty.INSTANCE, ThreadPool.Names.SAME + private class UpdateSnapshotStatusAction extends TransportMasterNodeAction< + UpdateIndexShardSnapshotStatusRequest, + ActionResponse.Empty> { + UpdateSnapshotStatusAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + UPDATE_SNAPSHOT_STATUS_ACTION_NAME, + false, + transportService, + clusterService, + threadPool, + actionFilters, + UpdateIndexShardSnapshotStatusRequest::new, + indexNameExpressionResolver, + in -> ActionResponse.Empty.INSTANCE, + ThreadPool.Names.SAME ); } @Override - protected void masterOperation(Task task, UpdateIndexShardSnapshotStatusRequest request, ClusterState state, - ActionListener listener) { - innerUpdateSnapshotState(new ShardSnapshotUpdate(request.snapshot(), request.shardId(), request.status()), - listener.map(v -> ActionResponse.Empty.INSTANCE)); + protected void masterOperation( + Task task, + UpdateIndexShardSnapshotStatusRequest request, + ClusterState state, + ActionListener listener + ) { + innerUpdateSnapshotState( + new ShardSnapshotUpdate(request.snapshot(), request.shardId(), request.status()), + listener.map(v -> ActionResponse.Empty.INSTANCE) + ); } @Override @@ -2909,8 +3254,10 @@ private final class FailPendingRepoTasksTask extends ClusterStateUpdateTask { @Override public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); boolean changed = false; final List remainingEntries = deletionsInProgress.getEntries(); List updatedEntries = new ArrayList<>(remainingEntries.size()); @@ -2923,8 +3270,7 @@ public ClusterState execute(ClusterState currentState) { } } final SnapshotDeletionsInProgress updatedDeletions = changed ? SnapshotDeletionsInProgress.of(updatedEntries) : null; - final SnapshotsInProgress snapshotsInProgress = - currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final List snapshotEntries = new ArrayList<>(); boolean changedSnapshots = false; for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { @@ -2945,20 +3291,26 @@ public ClusterState execute(ClusterState currentState) { @Override public void onFailure(String source, Exception e) { logger.info( - () -> new ParameterizedMessage("Failed to remove all snapshot tasks for repo [{}] from cluster state", repository), e); + () -> new ParameterizedMessage("Failed to remove all snapshot tasks for repo [{}] from cluster state", repository), + e + ); failAllListenersOnMasterFailOver(e); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - logger.warn(() -> - new ParameterizedMessage("Removed all snapshot tasks for repository [{}] from cluster state, now failing listeners", - repository), failure); + logger.warn( + () -> new ParameterizedMessage( + "Removed all snapshot tasks for repository [{}] from cluster state, now failing listeners", + repository + ), + failure + ); synchronized (currentlyFinalizing) { Tuple finalization; while ((finalization = repositoryOperations.pollFinalization(repository)) != null) { - assert snapshotsToFail.contains(finalization.v1().snapshot()) : - "[" + finalization.v1() + "] not found in snapshots to fail " + snapshotsToFail; + assert snapshotsToFail.contains(finalization.v1().snapshot()) + : "[" + finalization.v1() + "] not found in snapshots to fail " + snapshotsToFail; } leaveRepoLoop(repository); for (Snapshot snapshot : snapshotsToFail) { @@ -3039,15 +3391,16 @@ synchronized boolean isEmpty() { } synchronized boolean assertNotQueued(Snapshot snapshot) { - assert snapshotsToFinalize.getOrDefault(snapshot.getRepository(), new LinkedList<>()).stream() - .noneMatch(entry -> entry.snapshot().equals(snapshot)) : "Snapshot [" + snapshot + "] is still in finalization queue"; + assert snapshotsToFinalize.getOrDefault(snapshot.getRepository(), new LinkedList<>()) + .stream() + .noneMatch(entry -> entry.snapshot().equals(snapshot)) : "Snapshot [" + snapshot + "] is still in finalization queue"; return true; } synchronized boolean assertConsistent() { assert (latestKnownMetaData == null && snapshotsToFinalize.isEmpty()) - || (latestKnownMetaData != null && snapshotsToFinalize.isEmpty() == false) : - "Should not hold on to metadata if there are no more queued snapshots"; + || (latestKnownMetaData != null && snapshotsToFinalize.isEmpty() == false) + : "Should not hold on to metadata if there are no more queued snapshots"; assert snapshotsToFinalize.values().stream().noneMatch(Collection::isEmpty) : "Found empty queue in " + snapshotsToFinalize; return true; } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponseTests.java index 5deae29c302e0..6ab6a1c5c28e9 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponseTests.java @@ -13,7 +13,7 @@ import java.util.ArrayList; import java.util.List; -public class VerifyRepositoryResponseTests extends AbstractXContentTestCase { +public class VerifyRepositoryResponseTests extends AbstractXContentTestCase { @Override protected VerifyRepositoryResponse doParseInstance(XContentParser parser) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java index 0b02ef113104f..ce292e2755a8a 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotRequestTests.java @@ -66,7 +66,6 @@ public void testToXContent() throws IOException { original.featureStates(featureStates); } - if (randomBoolean()) { original.partial(randomBoolean()); } @@ -83,9 +82,12 @@ public void testToXContent() throws IOException { Collection wildcardStates = randomSubsetOf(Arrays.asList(WildcardStates.values())); Collection