-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Batch translog sync/upload per x ms for remote-backed indexes #5854
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
Changes from all commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
d5ba448
Batch translog upload per x ms to allow high index throughput
ashking94 4cfc52e
Add delay optimisations & log time taken for upload
ashking94 6d383df
Change log level to debug
ashking94 1f8736c
Move setting from Built-in to feature flag
ashking94 08a2879
Add UTs
ashking94 38f2016
Refactor code & change threadpool for translog sync
ashking94 ed67ae9
Add BufferedAsyncIOProcessorTests - UTs
ashking94 d76427e
Add UTs for BufferedAsyncIOProcessor
ashking94 8a14c89
Make buffer_interval setting Final
ashking94 e352565
Incorporate PR review feedback
ashking94 1f1ed7d
Incorporate PR review feedback
ashking94 70558bd
Incorporate PR review feedback
ashking94 3a5f960
Incorporate PR review feedback
ashking94 9dd5d88
Incorporate PR review feedback
ashking94 2ddda27
Incorporate PR review feedback
ashking94 d455c21
Handle scheduling failures
ashking94 105cbc5
Incorporate PR review feedback
ashking94 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
93 changes: 93 additions & 0 deletions
93
server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,93 @@ | ||
| /* | ||
| * SPDX-License-Identifier: Apache-2.0 | ||
| * | ||
| * The OpenSearch Contributors require contributions made to | ||
| * this file be licensed under the Apache-2.0 license or a | ||
| * compatible open source license. | ||
| */ | ||
|
|
||
| package org.opensearch.common.util.concurrent; | ||
|
|
||
| import org.apache.logging.log4j.Logger; | ||
| import org.opensearch.common.collect.Tuple; | ||
| import org.opensearch.common.unit.TimeValue; | ||
| import org.opensearch.threadpool.ThreadPool; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import java.util.Objects; | ||
| import java.util.function.Consumer; | ||
|
|
||
| /** | ||
| * A variant of {@link AsyncIOProcessor} that allows to batch and buffer processing items at every | ||
| * {@link BufferedAsyncIOProcessor#bufferInterval} in a separate threadpool. | ||
| * <p> | ||
| * Requests are buffered till processor thread calls @{link drainAndProcessAndRelease} after bufferInterval. | ||
| * If more requests are enqueued between invocations of drainAndProcessAndRelease, another processor thread | ||
| * gets scheduled. Subsequent requests will get buffered till drainAndProcessAndRelease gets called in this new | ||
| * processor thread. | ||
| * | ||
| * @opensearch.internal | ||
| */ | ||
| public abstract class BufferedAsyncIOProcessor<Item> extends AsyncIOProcessor<Item> { | ||
|
|
||
| private final ThreadPool threadpool; | ||
| private final TimeValue bufferInterval; | ||
|
|
||
| protected BufferedAsyncIOProcessor( | ||
| Logger logger, | ||
| int queueSize, | ||
| ThreadContext threadContext, | ||
| ThreadPool threadpool, | ||
| TimeValue bufferInterval | ||
| ) { | ||
| super(logger, queueSize, threadContext); | ||
| this.threadpool = threadpool; | ||
| this.bufferInterval = bufferInterval; | ||
| } | ||
|
|
||
| @Override | ||
| public void put(Item item, Consumer<Exception> listener) { | ||
| Objects.requireNonNull(item, "item must not be null"); | ||
| Objects.requireNonNull(listener, "listener must not be null"); | ||
| addToQueue(item, listener); | ||
| scheduleProcess(); | ||
| } | ||
|
|
||
| private void scheduleProcess() { | ||
| if (getQueue().isEmpty() == false && getPromiseSemaphore().tryAcquire()) { | ||
| try { | ||
| threadpool.schedule(this::process, getBufferInterval(), getBufferRefreshThreadPoolName()); | ||
| } catch (Exception e) { | ||
| getLogger().error("failed to schedule process"); | ||
| processSchedulingFailure(e); | ||
| getPromiseSemaphore().release(); | ||
| // This is to make sure that any new items that are added to the queue between processSchedulingFailure | ||
| // and releasing the semaphore is handled by a subsequent refresh and not starved. | ||
| scheduleProcess(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private void processSchedulingFailure(Exception e) { | ||
| List<Tuple<Item, Consumer<Exception>>> candidates = new ArrayList<>(); | ||
| getQueue().drainTo(candidates); | ||
| notifyList(candidates, e); | ||
| } | ||
|
|
||
| private void process() { | ||
| drainAndProcessAndRelease(new ArrayList<>()); | ||
| scheduleProcess(); | ||
ashking94 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| private TimeValue getBufferInterval() { | ||
| long timeSinceLastRunStartInNS = System.nanoTime() - getLastRunStartTimeInNs(); | ||
| if (timeSinceLastRunStartInNS >= bufferInterval.getNanos()) { | ||
| return TimeValue.ZERO; | ||
| } | ||
| return TimeValue.timeValueNanos(bufferInterval.getNanos() - timeSinceLastRunStartInNS); | ||
| } | ||
|
|
||
| protected abstract String getBufferRefreshThreadPoolName(); | ||
ashking94 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.