Skip to content

Rework shard snapshot workers #88209

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 74 commits into from
Sep 8, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
74 commits
Select commit Hold shift + click to select a range
031b2af
Fork off shard snapshot calls in parallel
pxsalehi Jun 30, 2022
7f40dc5
Update docs/changelog/88209.yaml
pxsalehi Jun 30, 2022
0184d38
temp
pxsalehi Jul 14, 2022
72f8624
Revert "temp"
pxsalehi Jul 14, 2022
4bfdfc7
Use queues and workers to handle shard snapshot work
pxsalehi Jul 15, 2022
b4fb408
cleanup, minor corrections
pxsalehi Jul 19, 2022
691de91
remove test logging
pxsalehi Jul 19, 2022
8fea5bf
Create only as many workers as needed
pxsalehi Jul 19, 2022
2213fa2
add test, minor renames
pxsalehi Jul 19, 2022
ea68db7
Add some asserts, simplify test
pxsalehi Jul 20, 2022
9dd7197
Edit changelog, remove redundant check
pxsalehi Jul 21, 2022
1b1e283
Merge branch 'master' into ps-fork-off-snapshot-calls
elasticmachine Jul 21, 2022
a44e47d
Move out cosmetic changes
pxsalehi Jul 22, 2022
35e039a
Merge branch 'ps-fork-off-snapshot-calls' of github.com:pxsalehi/elas…
pxsalehi Jul 22, 2022
21d4f92
Move out unrelated test change
pxsalehi Jul 22, 2022
6947ea1
Undo cosmetic changes to SnapshotShardsService
pxsalehi Jul 22, 2022
b073662
Merge remote-tracking branch 'upstream/master' into ps-fork-off-snaps…
pxsalehi Jul 22, 2022
05ed16c
Use a priority queue for both tasks
pxsalehi Jul 25, 2022
cfb47c3
Adapt tests
pxsalehi Jul 25, 2022
7710c89
Add test
pxsalehi Jul 25, 2022
4bbde07
Simplify mock
pxsalehi Jul 25, 2022
0c3a7c1
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Jul 25, 2022
fdb9cf3
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Jul 26, 2022
d67b2f9
Minor cleanup and rename
pxsalehi Jul 27, 2022
55bd708
One more assert
pxsalehi Jul 27, 2022
b291617
Always increment finishedShardSnapshotTasks
pxsalehi Jul 27, 2022
00f14a4
Merge branch 'main' into ps-fork-off-snapshot-calls
elasticmachine Jul 29, 2022
bd3b8b5
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 2, 2022
f220c1d
Package private size()
pxsalehi Aug 2, 2022
2e28c9b
Non-volatile worker count
pxsalehi Aug 2, 2022
ff90c49
Abstract class for snapshot tasks
pxsalehi Aug 2, 2022
5901020
Add to queue outside mutex
pxsalehi Aug 2, 2022
79acfd6
Simpler ensureEnoughWorkers
pxsalehi Aug 2, 2022
7ce5c41
Create workers in the constructor
pxsalehi Aug 2, 2022
5397f52
Minor renames
pxsalehi Aug 3, 2022
569913e
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 3, 2022
32a44f9
Do not use looping workers
pxsalehi Aug 3, 2022
a33aaeb
Use threadpool instead of mock in EncryptedRepositoryTests
pxsalehi Aug 3, 2022
12a408f
Undo changes in RepositoriesServiceTests
pxsalehi Aug 3, 2022
cd70f35
Use threadpool instead of mock in RepositoriesServiceTests
pxsalehi Aug 3, 2022
b7394e7
temp
pxsalehi Aug 4, 2022
9e003df
Revert "Use threadpool instead of mock in RepositoriesServiceTests"
pxsalehi Aug 4, 2022
8534cb2
Revert "Use threadpool instead of mock in EncryptedRepositoryTests"
pxsalehi Aug 4, 2022
1654166
add an executor to threadpool mock
pxsalehi Aug 4, 2022
54fbf89
Revert "temp"
pxsalehi Aug 4, 2022
e87b466
no executor, just info
pxsalehi Aug 4, 2022
3649729
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 4, 2022
8770046
Use lock-free instead of mutex
pxsalehi Aug 4, 2022
bbb65cd
Use startTime to order tasks
pxsalehi Aug 4, 2022
36f22e6
Cleanup
pxsalehi Aug 4, 2022
f6db134
More cleanup
pxsalehi Aug 4, 2022
2a16d07
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 8, 2022
a5784cc
Update changelog
pxsalehi Aug 8, 2022
0d0d70c
use random time in tests
pxsalehi Aug 11, 2022
fb4b217
Separate TaskRunner, address review
pxsalehi Aug 12, 2022
3ba20a3
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 25, 2022
a683832
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 26, 2022
613bcfb
minor comment changes
pxsalehi Aug 26, 2022
4f14c3e
Merge branch 'main' into ps-fork-off-snapshot-calls
elasticmachine Aug 29, 2022
a3bcbf0
Merge branch 'main' into ps-fork-off-snapshot-calls
elasticmachine Aug 29, 2022
ab7603f
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Aug 31, 2022
3a4a28e
Rename to PrioritizedThrottledTaskRunner
pxsalehi Aug 31, 2022
a4ef329
Comment methods only for testing
pxsalehi Aug 31, 2022
3c323f8
Use a name in the logger
pxsalehi Aug 31, 2022
09d10df
Use generic pool in test
pxsalehi Aug 31, 2022
865ceca
Rename test
pxsalehi Aug 31, 2022
ba0a411
Use threads not threadpool for test
pxsalehi Aug 31, 2022
56619c0
Add a priority to the TestTask
pxsalehi Aug 31, 2022
79331f8
Add more test and assertion
pxsalehi Sep 1, 2022
a752881
Sync threads in test
pxsalehi Sep 1, 2022
ed7ef03
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Sep 1, 2022
fc8f8df
Merge remote-tracking branch 'upstream/main' into ps-fork-off-snapsho…
pxsalehi Sep 6, 2022
4b6d83f
do not use Integer.compare
pxsalehi Sep 6, 2022
ad896fe
Address review comments
pxsalehi Sep 6, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions docs/changelog/88209.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
pr: 88209
summary: Prioritize shard snapshot tasks over file snapshot tasks and limit the number of the concurrently running snapshot tasks
area: Snapshot/Restore
type: enhancement
issues:
- 83408
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* 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.common.util.concurrent;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;

import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Executor;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;

/**
* {@link PrioritizedThrottledTaskRunner} performs the enqueued tasks in the order dictated by the
* natural ordering of the tasks, limiting the max number of concurrently running tasks. Each new task
* that is dequeued to be run, is forked off to the given executor.
*/
public class PrioritizedThrottledTaskRunner<T extends Comparable<T> & Runnable> {
private static final Logger logger = LogManager.getLogger(PrioritizedThrottledTaskRunner.class);

private final String taskRunnerName;
// The max number of tasks that this runner will schedule to concurrently run on the executor.
private final int maxRunningTasks;
// As we fork off dequeued tasks to the given executor, technically the following counter represents
// the number of the concurrent pollAndSpawn calls currently checking the queue for a task to run. This
// doesn't necessarily correspond to currently running tasks, since a pollAndSpawn could return without
// actually running a task when the queue is empty.
private final AtomicInteger runningTasks = new AtomicInteger();
private final BlockingQueue<T> tasks = new PriorityBlockingQueue<>();
private final Executor executor;

public PrioritizedThrottledTaskRunner(final String name, final int maxRunningTasks, final Executor executor) {
assert maxRunningTasks > 0;
this.taskRunnerName = name;
this.maxRunningTasks = maxRunningTasks;
this.executor = executor;
}

public void enqueueTask(final T task) {
logger.trace("[{}] enqueuing task {}", taskRunnerName, task);
tasks.add(task);
// Try to run a task since now there is at least one in the queue. If the maxRunningTasks is
// reached, the task is just enqueued.
pollAndSpawn();
}

// visible for testing
protected void pollAndSpawn() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add some commentary on how this loop works. In particular, on why we need to peek the queue. This is somewhat hard to follow for me and will be even harder to follow for future readers of this code.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point!

// A pollAndSpawn attempts to run a new task. There could be many concurrent pollAndSpawn calls competing
// to get a "free slot", since we attempt to run a new task on every enqueueTask call and every time an
// existing task is finished.
while (incrementRunningTasks()) {
T task = tasks.poll();
if (task == null) {
logger.trace("[{}] task queue is empty", taskRunnerName);
// We have taken up a "free slot", but there are no tasks in the queue! This could happen each time a worker
// sees an empty queue after running a task. Decrement to give competing pollAndSpawn calls a chance!
int decremented = runningTasks.decrementAndGet();
assert decremented >= 0;
// We might have blocked all competing pollAndSpawn calls. This could happen for example when
// maxRunningTasks=1 and a task got enqueued just after checking the queue but before decrementing.
// To be sure, return only if the queue is still empty. If the queue is not empty, this might be the
// only pollAndSpawn call in progress, and returning without peeking would risk ending up with a
// non-empty queue and no workers!
if (tasks.peek() == null) break;
} else {
executor.execute(() -> runTask(task));
}
}
}

// Each worker thread that runs a task, first needs to get a "free slot" in order to respect maxRunningTasks.
private boolean incrementRunningTasks() {
int preUpdateValue = runningTasks.getAndUpdate(v -> v < maxRunningTasks ? v + 1 : v);
assert preUpdateValue <= maxRunningTasks;
return preUpdateValue < maxRunningTasks;
}

// Only use for testing
public int runningTasks() {
return runningTasks.get();
}

// Only use for testing
public int queueSize() {
return tasks.size();
}

private void runTask(final T task) {
try {
logger.trace("[{}] running task {}", taskRunnerName, task);
task.run();
} finally {
// To avoid missing to run tasks that are enqueued and waiting, we check the queue again once running
// a task is finished.
int decremented = runningTasks.decrementAndGet();
assert decremented >= 0;
pollAndSpawn();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ public final class SnapshotShardContext extends ActionListener.Delegating<ShardS
private final IndexShardSnapshotStatus snapshotStatus;
private final Version repositoryMetaVersion;
private final Map<String, Object> userMetadata;
private final long snapshotStartTime;

/**
* @param store store to be snapshotted
Expand All @@ -51,6 +52,8 @@ public final class SnapshotShardContext extends ActionListener.Delegating<ShardS
* @param repositoryMetaVersion version of the updated repository metadata to write
* @param userMetadata user metadata of the snapshot found in
* {@link org.elasticsearch.cluster.SnapshotsInProgress.Entry#userMetadata()}
* @param snapshotStartTime start time of the snapshot found in
* {@link org.elasticsearch.cluster.SnapshotsInProgress.Entry#startTime()}
* @param listener listener invoked on completion
*/
public SnapshotShardContext(
Expand All @@ -63,6 +66,7 @@ public SnapshotShardContext(
IndexShardSnapshotStatus snapshotStatus,
Version repositoryMetaVersion,
Map<String, Object> userMetadata,
final long snapshotStartTime,
ActionListener<ShardSnapshotResult> listener
) {
super(ActionListener.runBefore(listener, commitRef::close));
Expand All @@ -75,6 +79,7 @@ public SnapshotShardContext(
this.snapshotStatus = snapshotStatus;
this.repositoryMetaVersion = repositoryMetaVersion;
this.userMetadata = userMetadata;
this.snapshotStartTime = snapshotStartTime;
}

public Store store() {
Expand Down Expand Up @@ -114,6 +119,10 @@ public Map<String, Object> userMetadata() {
return userMetadata;
}

public long snapshotStartTime() {
return snapshotStartTime;
}

@Override
public void onResponse(ShardSnapshotResult result) {
delegate.onResponse(result);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@
import java.util.stream.Stream;

import static org.elasticsearch.core.Strings.format;
import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo;
import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName;

/**
Expand Down Expand Up @@ -376,6 +377,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
*/
private final int maxSnapshotCount;

private final ShardSnapshotTaskRunner shardSnapshotTaskRunner;

/**
* Constructs new BlobStoreRepository
* @param metadata The metadata for this repository including name and settings
Expand Down Expand Up @@ -405,6 +408,12 @@ protected BlobStoreRepository(
this.basePath = basePath;
this.maxSnapshotCount = MAX_SNAPSHOTS_SETTING.get(metadata.settings());
this.repoDataDeduplicator = new ResultDeduplicator<>(threadPool.getThreadContext());
shardSnapshotTaskRunner = new ShardSnapshotTaskRunner(
threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(),
threadPool.executor(ThreadPool.Names.SNAPSHOT),
this::doSnapshotShard,
this::snapshotFile
);
}

@Override
Expand Down Expand Up @@ -2629,6 +2638,10 @@ private void writeAtomic(

@Override
public void snapshotShard(SnapshotShardContext context) {
shardSnapshotTaskRunner.enqueueShardSnapshot(context);
}

private void doSnapshotShard(SnapshotShardContext context) {
if (isReadOnly()) {
context.onFailure(new RepositoryException(metadata.name(), "cannot snapshot shard on a readonly repository"));
return;
Expand Down Expand Up @@ -2889,45 +2902,19 @@ public void snapshotShard(SnapshotShardContext context) {
snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis(), shardSnapshotResult);
context.onResponse(shardSnapshotResult);
}, context::onFailure);
if (indexIncrementalFileCount == 0) {
if (indexIncrementalFileCount == 0 || filesToSnapshot.isEmpty()) {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Turns out sometimes indexIncrementalFileCount can be non-zero, and filesToSnapshot still be empty. This was somehow not necessary to check before this change, since it would just lead to some idle workers that would exit immediately. However, with this change, we'd have to wait for actual file uploads to finish, not the file upload workers.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense, neat fix saving some redundant tasks :)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, with the current PR this was necessary, as otherwise, we'd be creating a GroupedActionListener of size 0.

allFilesUploadedListener.onResponse(Collections.emptyList());
return;
}
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
// 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(), indexIncrementalFileCount);
final ActionListener<Void> filesListener = fileQueueListener(filesToSnapshot, workers, allFilesUploadedListener);
for (int i = 0; i < workers; ++i) {
executeOneFileSnapshot(store, snapshotId, context.indexId(), snapshotStatus, filesToSnapshot, executor, filesListener);
final ActionListener<Void> filesListener = fileQueueListener(filesToSnapshot, filesToSnapshot.size(), allFilesUploadedListener);
for (FileInfo fileInfo : filesToSnapshot) {
shardSnapshotTaskRunner.enqueueFileSnapshot(context, fileInfo, filesListener);
}
} catch (Exception e) {
context.onFailure(e);
}
}

private void executeOneFileSnapshot(
Store store,
SnapshotId snapshotId,
IndexId indexId,
IndexShardSnapshotStatus snapshotStatus,
BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot,
Executor executor,
ActionListener<Void> listener
) throws InterruptedException {
final ShardId shardId = store.shardId();
final BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = filesToSnapshot.poll(0L, TimeUnit.MILLISECONDS);
if (snapshotFileInfo == null) {
listener.onResponse(null);
} else {
executor.execute(ActionRunnable.wrap(listener, l -> {
try (Releasable ignored = incrementStoreRef(store, snapshotStatus, shardId)) {
snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store);
executeOneFileSnapshot(store, snapshotId, indexId, snapshotStatus, filesToSnapshot, executor, l);
}
}));
}
}

private static Releasable incrementStoreRef(Store store, IndexShardSnapshotStatus snapshotStatus, ShardId shardId) {
if (store.tryIncRef() == false) {
if (snapshotStatus.isAborted()) {
Expand Down Expand Up @@ -3116,10 +3103,10 @@ void ensureNotClosing(final Store store) throws AlreadyClosedException {

private static ActionListener<Void> fileQueueListener(
BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> files,
int workers,
int numberOfFiles,
ActionListener<Collection<Void>> listener
) {
return new GroupedActionListener<>(listener, workers).delegateResponse((l, e) -> {
return new GroupedActionListener<>(listener, numberOfFiles).delegateResponse((l, e) -> {
files.clear(); // Stop uploading the remaining files if we run into any exception
l.onFailure(e);
});
Expand Down Expand Up @@ -3426,19 +3413,20 @@ private Tuple<BlobStoreIndexShardSnapshots, Long> buildBlobStoreIndexShardSnapsh

/**
* Snapshot individual file
* @param fileInfo file to be snapshotted
* @param fileInfo file to snapshot
*/
private void snapshotFile(
BlobStoreIndexShardSnapshot.FileInfo fileInfo,
IndexId indexId,
ShardId shardId,
SnapshotId snapshotId,
IndexShardSnapshotStatus snapshotStatus,
Store store
) throws IOException {
private void snapshotFile(SnapshotShardContext context, FileInfo fileInfo) throws IOException {
final IndexId indexId = context.indexId();
final Store store = context.store();
final ShardId shardId = store.shardId();
final IndexShardSnapshotStatus snapshotStatus = context.status();
final SnapshotId snapshotId = context.snapshotId();
final BlobContainer shardContainer = shardContainer(indexId, shardId);
final String file = fileInfo.physicalName();
try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) {
try (
Releasable ignored = BlobStoreRepository.incrementStoreRef(store, snapshotStatus, store.shardId());
IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())
) {
for (int i = 0; i < fileInfo.numberOfParts(); i++) {
final long partBytes = fileInfo.partBytes(i);

Expand Down
Loading