Skip to content

Commit dbb626a

Browse files
Add Bulk Fetch SnapshotInfo API to Repository (#73570)
This PR refactors the `Repository` API for fetching `SnapshotInfo` to enabled implementations to optimize for bulk fetching multiple `SnapshotInfo` at once. This is a requirement for making use of a more efficient repository format that does not require loading individual blobs per snapshot to fetch a snapshot listing. Also, by enabling consuming `SnapshotInfo` as they are fetched on the snapshot meta thread this allows for some more memory efficient usage of snapshot listing. Also, this commit makes use of the new API to make the snapshot status API run a little more parallel if fetching multiple snapshots (though there's additional improvements possible+useful here as far as fetching shard level metadata in parallel).
1 parent 80593b4 commit dbb626a

File tree

13 files changed

+478
-213
lines changed

13 files changed

+478
-213
lines changed

server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java

Lines changed: 22 additions & 58 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@
1010

1111
import org.apache.logging.log4j.LogManager;
1212
import org.apache.logging.log4j.Logger;
13-
import org.apache.logging.log4j.message.ParameterizedMessage;
1413
import org.apache.lucene.util.CollectionUtil;
1514
import org.elasticsearch.ElasticsearchException;
1615
import org.elasticsearch.action.ActionListener;
@@ -29,12 +28,12 @@
2928
import org.elasticsearch.core.Nullable;
3029
import org.elasticsearch.common.inject.Inject;
3130
import org.elasticsearch.common.regex.Regex;
31+
import org.elasticsearch.repositories.GetSnapshotInfoContext;
3232
import org.elasticsearch.repositories.IndexId;
3333
import org.elasticsearch.repositories.RepositoriesService;
3434
import org.elasticsearch.repositories.Repository;
3535
import org.elasticsearch.repositories.RepositoryData;
3636
import org.elasticsearch.repositories.RepositoryMissingException;
37-
import org.elasticsearch.snapshots.SnapshotException;
3837
import org.elasticsearch.snapshots.SnapshotId;
3938
import org.elasticsearch.snapshots.SnapshotInfo;
4039
import org.elasticsearch.snapshots.SnapshotMissingException;
@@ -53,8 +52,6 @@
5352
import java.util.List;
5453
import java.util.Map;
5554
import java.util.Set;
56-
import java.util.concurrent.BlockingQueue;
57-
import java.util.concurrent.LinkedBlockingQueue;
5855
import java.util.stream.Collectors;
5956

6057
import static java.util.Collections.unmodifiableList;
@@ -327,76 +324,43 @@ private void snapshots(
327324
} else {
328325
snapshotInfos = Collections.synchronizedList(new ArrayList<>());
329326
}
330-
final ActionListener<Collection<Void>> allDoneListener = listener.delegateFailure((l, v) -> {
327+
final ActionListener<Void> allDoneListener = listener.delegateFailure((l, v) -> {
331328
final ArrayList<SnapshotInfo> snapshotList = new ArrayList<>(snapshotInfos);
332329
snapshotList.addAll(snapshotSet);
333330
CollectionUtil.timSort(snapshotList);
334331
listener.onResponse(unmodifiableList(snapshotList));
335332
});
336333
if (snapshotIdsToIterate.isEmpty()) {
337-
allDoneListener.onResponse(Collections.emptyList());
334+
allDoneListener.onResponse(null);
338335
return;
339336
}
340-
// put snapshot info downloads into a task queue instead of pushing them all into the queue to not completely monopolize the
341-
// snapshot meta pool for a single request
342-
final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_META).getMax(), snapshotIdsToIterate.size());
343-
final BlockingQueue<SnapshotId> queue = new LinkedBlockingQueue<>(snapshotIdsToIterate);
344-
final ActionListener<Void> workerDoneListener = new GroupedActionListener<>(allDoneListener, workers).delegateResponse((l, e) -> {
345-
queue.clear(); // Stop fetching the remaining snapshots once we've failed fetching one since the response is an error response
346-
// anyway in this case
347-
l.onFailure(e);
348-
});
349337
final Repository repository;
350338
try {
351339
repository = repositoriesService.repository(repositoryName);
352340
} catch (RepositoryMissingException e) {
353341
listener.onFailure(e);
354342
return;
355343
}
356-
for (int i = 0; i < workers; i++) {
357-
getOneSnapshotInfo(ignoreUnavailable, repository, queue, snapshotInfos, task, workerDoneListener);
358-
}
359-
}
344+
repository.getSnapshotInfo(
345+
new GetSnapshotInfoContext(
346+
snapshotIdsToIterate,
347+
ignoreUnavailable == false,
348+
task::isCancelled,
349+
(context, snapshotInfo) -> snapshotInfos.add(snapshotInfo),
350+
ignoreUnavailable ? ActionListener.runAfter(new ActionListener<>() {
351+
@Override
352+
public void onResponse(Void unused) {
353+
logger.trace("done fetching snapshot infos [{}]", snapshotIdsToIterate);
354+
}
360355

361-
/**
362-
* Tries to poll a {@link SnapshotId} to load {@link SnapshotInfo} for from the given {@code queue}. If it finds one in the queue,
363-
* loads the snapshot info from the repository and adds it to the given {@code snapshotInfos} collection, then invokes itself again to
364-
* try and poll another task from the queue.
365-
* If the queue is empty resolves {@code} listener.
366-
*/
367-
private void getOneSnapshotInfo(
368-
boolean ignoreUnavailable,
369-
Repository repository,
370-
BlockingQueue<SnapshotId> queue,
371-
Collection<SnapshotInfo> snapshotInfos,
372-
CancellableTask task,
373-
ActionListener<Void> listener
374-
) {
375-
final SnapshotId snapshotId = queue.poll();
376-
if (snapshotId == null) {
377-
listener.onResponse(null);
378-
return;
379-
}
380-
threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> {
381-
if (task.isCancelled()) {
382-
listener.onFailure(new TaskCancelledException("task cancelled"));
383-
return;
384-
}
385-
try {
386-
snapshotInfos.add(repository.getSnapshotInfo(snapshotId));
387-
} catch (Exception ex) {
388-
if (ignoreUnavailable) {
389-
logger.warn(() -> new ParameterizedMessage("failed to get snapshot [{}]", snapshotId), ex);
390-
} else {
391-
listener.onFailure(
392-
ex instanceof SnapshotException
393-
? ex
394-
: new SnapshotException(repository.getMetadata().name(), snapshotId, "Snapshot could not be read", ex)
395-
);
396-
}
397-
}
398-
getOneSnapshotInfo(ignoreUnavailable, repository, queue, snapshotInfos, task, listener);
399-
});
356+
@Override
357+
public void onFailure(Exception e) {
358+
assert false : new AssertionError("listener should always complete successfully for ignoreUnavailable=true", e);
359+
logger.warn("failed to fetch snapshot info for some snapshots", e);
360+
}
361+
}, () -> allDoneListener.onResponse(null)) : allDoneListener
362+
)
363+
);
400364
}
401365

402366
private boolean isAllSnapshots(String[] snapshots) {

server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java

Lines changed: 70 additions & 78 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@
1313
import org.apache.logging.log4j.LogManager;
1414
import org.apache.logging.log4j.Logger;
1515
import org.elasticsearch.action.ActionListener;
16-
import org.elasticsearch.action.ActionRunnable;
16+
import org.elasticsearch.action.StepListener;
1717
import org.elasticsearch.action.support.ActionFilters;
1818
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
1919
import org.elasticsearch.client.node.NodeClient;
@@ -27,10 +27,10 @@
2727
import org.elasticsearch.common.Strings;
2828
import org.elasticsearch.common.inject.Inject;
2929
import org.elasticsearch.common.util.CollectionUtils;
30-
import org.elasticsearch.common.util.concurrent.ListenableFuture;
3130
import org.elasticsearch.common.util.set.Sets;
3231
import org.elasticsearch.index.shard.ShardId;
3332
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
33+
import org.elasticsearch.repositories.GetSnapshotInfoContext;
3434
import org.elasticsearch.repositories.IndexId;
3535
import org.elasticsearch.repositories.RepositoriesService;
3636
import org.elasticsearch.repositories.Repository;
@@ -52,6 +52,7 @@
5252
import java.io.IOException;
5353
import java.util.ArrayList;
5454
import java.util.Arrays;
55+
import java.util.Collection;
5556
import java.util.Collections;
5657
import java.util.HashMap;
5758
import java.util.HashSet;
@@ -62,6 +63,7 @@
6263
import java.util.stream.Collectors;
6364

6465
import static java.util.Collections.unmodifiableMap;
66+
import static org.elasticsearch.cluster.SnapshotsInProgress.ShardState.SUCCESS;
6567

6668
public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<SnapshotsStatusRequest, SnapshotsStatusResponse> {
6769

@@ -90,7 +92,7 @@ public TransportSnapshotsStatusAction(
9092
SnapshotsStatusRequest::new,
9193
indexNameExpressionResolver,
9294
SnapshotsStatusResponse::new,
93-
ThreadPool.Names.GENERIC
95+
ThreadPool.Names.SAME
9496
);
9597
this.repositoriesService = repositoriesService;
9698
this.client = client;
@@ -142,13 +144,14 @@ protected void masterOperation(
142144
new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)).snapshots(snapshots)
143145
.timeout(request.masterNodeTimeout()),
144146
ActionListener.wrap(
145-
nodeSnapshotStatuses -> threadPool.generic()
146-
.execute(
147-
ActionRunnable.wrap(
148-
listener,
149-
l -> buildResponse(snapshotsInProgress, request, currentSnapshots, nodeSnapshotStatuses, cancellableTask, l)
150-
)
151-
),
147+
nodeSnapshotStatuses -> buildResponse(
148+
snapshotsInProgress,
149+
request,
150+
currentSnapshots,
151+
nodeSnapshotStatuses,
152+
cancellableTask,
153+
listener
154+
),
152155
listener::onFailure
153156
)
154157
);
@@ -192,8 +195,7 @@ private void buildResponse(
192195
SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.key);
193196
if (shardStatus != null) {
194197
// We have full information about this shard
195-
if (shardStatus.getStage() == SnapshotIndexShardStage.DONE
196-
&& shardEntry.value.state() != SnapshotsInProgress.ShardState.SUCCESS) {
198+
if (shardStatus.getStage() == SnapshotIndexShardStage.DONE && shardEntry.value.state() != SUCCESS) {
197199
// Unlikely edge case:
198200
// Data node has finished snapshotting the shard but the cluster state has not yet been updated
199201
// to reflect this. We adjust the status to show up as snapshot metadata being written because
@@ -286,9 +288,10 @@ private void loadRepositoryData(
286288
ActionListener<SnapshotsStatusResponse> listener
287289
) {
288290
final Set<String> requestedSnapshotNames = Sets.newHashSet(request.snapshots());
289-
final ListenableFuture<RepositoryData> repositoryDataListener = new ListenableFuture<>();
291+
final StepListener<RepositoryData> repositoryDataListener = new StepListener<>();
290292
repositoriesService.getRepositoryData(repositoryName, repositoryDataListener);
291-
repositoryDataListener.addListener(ActionListener.wrap(repositoryData -> {
293+
final Collection<SnapshotId> snapshotIdsToLoad = new ArrayList<>();
294+
repositoryDataListener.whenComplete(repositoryData -> {
292295
ensureNotCancelled(task);
293296
final Map<String, SnapshotId> matchedSnapshotIds = repositoryData.getSnapshotIds()
294297
.stream()
@@ -314,73 +317,62 @@ private void loadRepositoryData(
314317
throw new SnapshotMissingException(repositoryName, snapshotName);
315318
}
316319
}
317-
SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId);
318-
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
319-
if (snapshotInfo.state().completed()) {
320-
Map<ShardId, IndexShardSnapshotStatus> shardStatuses = snapshotShards(
321-
repositoryName,
322-
repositoryData,
323-
task,
324-
snapshotInfo
325-
);
326-
for (Map.Entry<ShardId, IndexShardSnapshotStatus> shardStatus : shardStatuses.entrySet()) {
327-
IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy();
328-
shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus));
329-
}
330-
final SnapshotsInProgress.State state;
331-
switch (snapshotInfo.state()) {
332-
case FAILED:
333-
state = SnapshotsInProgress.State.FAILED;
334-
break;
335-
case SUCCESS:
336-
case PARTIAL:
337-
// Translating both PARTIAL and SUCCESS to SUCCESS for now
338-
// TODO: add the differentiation on the metadata level in the next major release
339-
state = SnapshotsInProgress.State.SUCCESS;
340-
break;
341-
default:
342-
throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state());
343-
}
344-
final long startTime = snapshotInfo.startTime();
345-
final long endTime = snapshotInfo.endTime();
346-
assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false)
347-
: "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]";
348-
builder.add(
349-
new SnapshotStatus(
350-
new Snapshot(repositoryName, snapshotId),
351-
state,
352-
Collections.unmodifiableList(shardStatusBuilder),
353-
snapshotInfo.includeGlobalState(),
354-
startTime,
355-
// Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0
356-
(endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime
357-
)
358-
);
320+
if (snapshotsInProgress.snapshot(new Snapshot(repositoryName, snapshotId)) == null) {
321+
snapshotIdsToLoad.add(snapshotId);
359322
}
360323
}
361-
listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder)));
362-
}, listener::onFailure), threadPool.generic(), null);
363-
}
364324

365-
/**
366-
* Retrieves snapshot from repository
367-
*
368-
* @param snapshotsInProgress snapshots in progress in the cluster state
369-
* @param repositoryName repository name
370-
* @param snapshotId snapshot id
371-
* @return snapshot
372-
* @throws SnapshotMissingException if snapshot is not found
373-
*/
374-
private SnapshotInfo snapshot(SnapshotsInProgress snapshotsInProgress, String repositoryName, SnapshotId snapshotId) {
375-
List<SnapshotsInProgress.Entry> entries = SnapshotsService.currentSnapshots(
376-
snapshotsInProgress,
377-
repositoryName,
378-
Collections.singletonList(snapshotId.getName())
379-
);
380-
if (entries.isEmpty() == false) {
381-
return new SnapshotInfo(entries.iterator().next());
382-
}
383-
return repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotId);
325+
if (snapshotIdsToLoad.isEmpty()) {
326+
listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder)));
327+
} else {
328+
final List<SnapshotStatus> threadSafeBuilder = Collections.synchronizedList(builder);
329+
repositoriesService.repository(repositoryName)
330+
.getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, task::isCancelled, (context, snapshotInfo) -> {
331+
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();
332+
final Map<ShardId, IndexShardSnapshotStatus> shardStatuses;
333+
try {
334+
shardStatuses = snapshotShards(repositoryName, repositoryData, task, snapshotInfo);
335+
} catch (Exception e) {
336+
// stops all further fetches of snapshotInfo since context is fail-fast
337+
context.onFailure(e);
338+
return;
339+
}
340+
for (Map.Entry<ShardId, IndexShardSnapshotStatus> shardStatus : shardStatuses.entrySet()) {
341+
IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy();
342+
shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus));
343+
}
344+
final SnapshotsInProgress.State state;
345+
switch (snapshotInfo.state()) {
346+
case FAILED:
347+
state = SnapshotsInProgress.State.FAILED;
348+
break;
349+
case SUCCESS:
350+
case PARTIAL:
351+
// Translating both PARTIAL and SUCCESS to SUCCESS for now
352+
// TODO: add the differentiation on the metadata level in the next major release
353+
state = SnapshotsInProgress.State.SUCCESS;
354+
break;
355+
default:
356+
throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state());
357+
}
358+
final long startTime = snapshotInfo.startTime();
359+
final long endTime = snapshotInfo.endTime();
360+
assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false)
361+
: "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]";
362+
threadSafeBuilder.add(
363+
new SnapshotStatus(
364+
new Snapshot(repositoryName, snapshotInfo.snapshotId()),
365+
state,
366+
Collections.unmodifiableList(shardStatusBuilder),
367+
snapshotInfo.includeGlobalState(),
368+
startTime,
369+
// Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0
370+
(endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime
371+
)
372+
);
373+
}, listener.map(v -> new SnapshotsStatusResponse(List.copyOf(threadSafeBuilder)))));
374+
}
375+
}, listener::onFailure);
384376
}
385377

386378
/**

server/src/main/java/org/elasticsearch/repositories/FilterRepository.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,8 +49,8 @@ public RepositoryMetadata getMetadata() {
4949
}
5050

5151
@Override
52-
public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) {
53-
return in.getSnapshotInfo(snapshotId);
52+
public void getSnapshotInfo(GetSnapshotInfoContext context) {
53+
in.getSnapshotInfo(context);
5454
}
5555

5656
@Override

0 commit comments

Comments
 (0)