Skip to content

Commit a8de554

Browse files
authored
Refactor merge scheduling code to allow overrides (#114547)
This code refactors how the merge scheduler is configured to allow different engine implementations to configure different merge schedulers.
1 parent 35fd893 commit a8de554

File tree

5 files changed

+226
-120
lines changed

5 files changed

+226
-120
lines changed

server/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java

Lines changed: 21 additions & 91 deletions
Original file line numberDiff line numberDiff line change
@@ -15,11 +15,7 @@
1515
import org.apache.lucene.index.MergeScheduler;
1616
import org.apache.lucene.util.SameThreadExecutorService;
1717
import org.elasticsearch.common.logging.Loggers;
18-
import org.elasticsearch.common.metrics.CounterMetric;
19-
import org.elasticsearch.common.metrics.MeanMetric;
2018
import org.elasticsearch.common.settings.Settings;
21-
import org.elasticsearch.common.unit.ByteSizeValue;
22-
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
2319
import org.elasticsearch.common.util.concurrent.EsExecutors;
2420
import org.elasticsearch.core.TimeValue;
2521
import org.elasticsearch.index.IndexSettings;
@@ -29,32 +25,20 @@
2925
import org.elasticsearch.index.shard.ShardId;
3026

3127
import java.io.IOException;
32-
import java.util.Collections;
33-
import java.util.Locale;
3428
import java.util.Set;
3529
import java.util.concurrent.Executor;
3630

3731
/**
3832
* An extension to the {@link ConcurrentMergeScheduler} that provides tracking on merge times, total
3933
* and current merges.
4034
*/
41-
class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
35+
public class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler implements ElasticsearchMergeScheduler {
4236

4337
protected final Logger logger;
4438
private final Settings indexSettings;
4539
private final ShardId shardId;
4640

47-
private final MeanMetric totalMerges = new MeanMetric();
48-
private final CounterMetric totalMergesNumDocs = new CounterMetric();
49-
private final CounterMetric totalMergesSizeInBytes = new CounterMetric();
50-
private final CounterMetric currentMerges = new CounterMetric();
51-
private final CounterMetric currentMergesNumDocs = new CounterMetric();
52-
private final CounterMetric currentMergesSizeInBytes = new CounterMetric();
53-
private final CounterMetric totalMergeStoppedTime = new CounterMetric();
54-
private final CounterMetric totalMergeThrottledTime = new CounterMetric();
55-
56-
private final Set<OnGoingMerge> onGoingMerges = ConcurrentCollections.newConcurrentSet();
57-
private final Set<OnGoingMerge> readOnlyOnGoingMerges = Collections.unmodifiableSet(onGoingMerges);
41+
private final MergeTracking mergeTracking;
5842
private final MergeSchedulerConfig config;
5943
private final SameThreadExecutorService sameThreadExecutorService = new SameThreadExecutorService();
6044

@@ -63,11 +47,16 @@ class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
6347
this.shardId = shardId;
6448
this.indexSettings = indexSettings.getSettings();
6549
this.logger = Loggers.getLogger(getClass(), shardId);
50+
this.mergeTracking = new MergeTracking(
51+
logger,
52+
() -> indexSettings.getMergeSchedulerConfig().isAutoThrottle() ? getIORateLimitMBPerSec() : Double.POSITIVE_INFINITY
53+
);
6654
refreshConfig();
6755
}
6856

57+
@Override
6958
public Set<OnGoingMerge> onGoingMerges() {
70-
return readOnlyOnGoingMerges;
59+
return mergeTracking.onGoingMerges();
7160
}
7261

7362
/** We're currently only interested in messages with this prefix. */
@@ -104,74 +93,21 @@ protected void message(String message) {
10493
super.message(message);
10594
}
10695

107-
private static String getSegmentName(MergePolicy.OneMerge merge) {
108-
return merge.getMergeInfo() != null ? merge.getMergeInfo().info.name : "_na_";
109-
}
110-
11196
@Override
11297
protected void doMerge(MergeSource mergeSource, MergePolicy.OneMerge merge) throws IOException {
113-
int totalNumDocs = merge.totalNumDocs();
114-
long totalSizeInBytes = merge.totalBytesSize();
11598
long timeNS = System.nanoTime();
116-
currentMerges.inc();
117-
currentMergesNumDocs.inc(totalNumDocs);
118-
currentMergesSizeInBytes.inc(totalSizeInBytes);
119-
12099
OnGoingMerge onGoingMerge = new OnGoingMerge(merge);
121-
onGoingMerges.add(onGoingMerge);
122-
123-
if (logger.isTraceEnabled()) {
124-
logger.trace(
125-
"merge [{}] starting..., merging [{}] segments, [{}] docs, [{}] size, into [{}] estimated_size",
126-
getSegmentName(merge),
127-
merge.segments.size(),
128-
totalNumDocs,
129-
ByteSizeValue.ofBytes(totalSizeInBytes),
130-
ByteSizeValue.ofBytes(merge.estimatedMergeBytes)
131-
);
132-
}
100+
mergeTracking.mergeStarted(onGoingMerge);
133101
try {
134102
beforeMerge(onGoingMerge);
135103
super.doMerge(mergeSource, merge);
136104
} finally {
137105
long tookMS = TimeValue.nsecToMSec(System.nanoTime() - timeNS);
106+
mergeTracking.mergeFinished(merge, onGoingMerge, tookMS);
138107

139-
onGoingMerges.remove(onGoingMerge);
140108
afterMerge(onGoingMerge);
141-
142-
currentMerges.dec();
143-
currentMergesNumDocs.dec(totalNumDocs);
144-
currentMergesSizeInBytes.dec(totalSizeInBytes);
145-
146-
totalMergesNumDocs.inc(totalNumDocs);
147-
totalMergesSizeInBytes.inc(totalSizeInBytes);
148-
totalMerges.inc(tookMS);
149-
long stoppedMS = TimeValue.nsecToMSec(
150-
merge.getMergeProgress().getPauseTimes().get(MergePolicy.OneMergeProgress.PauseReason.STOPPED)
151-
);
152-
long throttledMS = TimeValue.nsecToMSec(
153-
merge.getMergeProgress().getPauseTimes().get(MergePolicy.OneMergeProgress.PauseReason.PAUSED)
154-
);
155-
totalMergeStoppedTime.inc(stoppedMS);
156-
totalMergeThrottledTime.inc(throttledMS);
157-
158-
String message = String.format(
159-
Locale.ROOT,
160-
"merge segment [%s] done: took [%s], [%,.1f MB], [%,d docs], [%s stopped], [%s throttled]",
161-
getSegmentName(merge),
162-
TimeValue.timeValueMillis(tookMS),
163-
totalSizeInBytes / 1024f / 1024f,
164-
totalNumDocs,
165-
TimeValue.timeValueMillis(stoppedMS),
166-
TimeValue.timeValueMillis(throttledMS)
167-
);
168-
169-
if (tookMS > 20000) { // if more than 20 seconds, DEBUG log it
170-
logger.debug("{}", message);
171-
} else if (logger.isTraceEnabled()) {
172-
logger.trace("{}", message);
173-
}
174109
}
110+
175111
}
176112

177113
/**
@@ -206,24 +142,13 @@ protected MergeThread getMergeThread(MergeSource mergeSource, MergePolicy.OneMer
206142
return thread;
207143
}
208144

209-
MergeStats stats() {
210-
final MergeStats mergeStats = new MergeStats();
211-
mergeStats.add(
212-
totalMerges.count(),
213-
totalMerges.sum(),
214-
totalMergesNumDocs.count(),
215-
totalMergesSizeInBytes.count(),
216-
currentMerges.count(),
217-
currentMergesNumDocs.count(),
218-
currentMergesSizeInBytes.count(),
219-
totalMergeStoppedTime.count(),
220-
totalMergeThrottledTime.count(),
221-
config.isAutoThrottle() ? getIORateLimitMBPerSec() : Double.POSITIVE_INFINITY
222-
);
223-
return mergeStats;
145+
@Override
146+
public MergeStats stats() {
147+
return mergeTracking.stats();
224148
}
225149

226-
void refreshConfig() {
150+
@Override
151+
public void refreshConfig() {
227152
if (this.getMaxMergeCount() != config.getMaxMergeCount() || this.getMaxThreadCount() != config.getMaxThreadCount()) {
228153
this.setMaxMergesAndThreads(config.getMaxMergeCount(), config.getMaxThreadCount());
229154
}
@@ -234,4 +159,9 @@ void refreshConfig() {
234159
disableAutoIOThrottle();
235160
}
236161
}
162+
163+
@Override
164+
public MergeScheduler getMergeScheduler() {
165+
return this;
166+
}
237167
}
Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
/*
2+
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
3+
* or more contributor license agreements. Licensed under the "Elastic License
4+
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
5+
* Public License v 1"; you may not use this file except in compliance with, at
6+
* your election, the "Elastic License 2.0", the "GNU Affero General Public
7+
* License v3.0 only", or the "Server Side Public License, v 1".
8+
*/
9+
10+
package org.elasticsearch.index.engine;
11+
12+
import org.apache.lucene.index.MergeScheduler;
13+
import org.elasticsearch.index.merge.MergeStats;
14+
import org.elasticsearch.index.merge.OnGoingMerge;
15+
16+
import java.util.Set;
17+
18+
public interface ElasticsearchMergeScheduler {
19+
20+
Set<OnGoingMerge> onGoingMerges();
21+
22+
MergeStats stats();
23+
24+
void refreshConfig();
25+
26+
MergeScheduler getMergeScheduler();
27+
}

server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java

Lines changed: 39 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import org.apache.lucene.index.LeafReaderContext;
2121
import org.apache.lucene.index.LiveIndexWriterConfig;
2222
import org.apache.lucene.index.MergePolicy;
23+
import org.apache.lucene.index.MergeScheduler;
2324
import org.apache.lucene.index.SegmentCommitInfo;
2425
import org.apache.lucene.index.SegmentInfos;
2526
import org.apache.lucene.index.SoftDeletesRetentionMergePolicy;
@@ -139,7 +140,7 @@ public class InternalEngine extends Engine {
139140
private volatile long lastDeleteVersionPruneTimeMSec;
140141

141142
private final Translog translog;
142-
private final ElasticsearchConcurrentMergeScheduler mergeScheduler;
143+
private final ElasticsearchMergeScheduler mergeScheduler;
143144

144145
private final IndexWriter indexWriter;
145146

@@ -248,11 +249,12 @@ public InternalEngine(EngineConfig engineConfig) {
248249
Translog translog = null;
249250
ExternalReaderManager externalReaderManager = null;
250251
ElasticsearchReaderManager internalReaderManager = null;
251-
EngineMergeScheduler scheduler = null;
252+
MergeScheduler scheduler = null;
252253
boolean success = false;
253254
try {
254255
this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().relativeTimeInMillis();
255-
mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings());
256+
mergeScheduler = createMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings());
257+
scheduler = mergeScheduler.getMergeScheduler();
256258
throttle = new IndexThrottle();
257259
try {
258260
store.trimUnsafeCommits(config().getTranslogConfig().getTranslogPath());
@@ -383,7 +385,7 @@ private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException {
383385

384386
@Nullable
385387
private CombinedDeletionPolicy.CommitsListener newCommitsListener() {
386-
Engine.IndexCommitListener listener = engineConfig.getIndexCommitListener();
388+
IndexCommitListener listener = engineConfig.getIndexCommitListener();
387389
if (listener != null) {
388390
final IndexCommitListener wrappedListener = Assertions.ENABLED ? assertingCommitsOrderListener(listener) : listener;
389391
return new CombinedDeletionPolicy.CommitsListener() {
@@ -824,7 +826,7 @@ private GetResult getFromTranslog(
824826
config(),
825827
translogInMemorySegmentsCount::incrementAndGet
826828
);
827-
final Engine.Searcher searcher = new Engine.Searcher(
829+
final Searcher searcher = new Searcher(
828830
"realtime_get",
829831
ElasticsearchDirectoryReader.wrap(inMemoryReader, shardId),
830832
config().getSimilarity(),
@@ -841,7 +843,7 @@ public GetResult get(
841843
Get get,
842844
MappingLookup mappingLookup,
843845
DocumentParser documentParser,
844-
Function<Engine.Searcher, Engine.Searcher> searcherWrapper
846+
Function<Searcher, Searcher> searcherWrapper
845847
) {
846848
try (var ignored = acquireEnsureOpenRef()) {
847849
if (get.realtime()) {
@@ -875,7 +877,7 @@ protected GetResult realtimeGetUnderLock(
875877
Get get,
876878
MappingLookup mappingLookup,
877879
DocumentParser documentParser,
878-
Function<Engine.Searcher, Engine.Searcher> searcherWrapper,
880+
Function<Searcher, Searcher> searcherWrapper,
879881
boolean getFromSearcher
880882
) {
881883
assert isDrainedForClose() == false;
@@ -1098,7 +1100,7 @@ protected boolean assertPrimaryCanOptimizeAddDocument(final Index index) {
10981100
return true;
10991101
}
11001102

1101-
private boolean assertIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) {
1103+
private boolean assertIncomingSequenceNumber(final Operation.Origin origin, final long seqNo) {
11021104
if (origin == Operation.Origin.PRIMARY) {
11031105
assert assertPrimaryIncomingSequenceNumber(origin, seqNo);
11041106
} else {
@@ -1108,7 +1110,7 @@ private boolean assertIncomingSequenceNumber(final Engine.Operation.Origin origi
11081110
return true;
11091111
}
11101112

1111-
protected boolean assertPrimaryIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) {
1113+
protected boolean assertPrimaryIncomingSequenceNumber(final Operation.Origin origin, final long seqNo) {
11121114
// sequence number should not be set when operation origin is primary
11131115
assert seqNo == SequenceNumbers.UNASSIGNED_SEQ_NO
11141116
: "primary operations must never have an assigned sequence number but was [" + seqNo + "]";
@@ -2700,7 +2702,7 @@ private IndexWriterConfig getIndexWriterConfig() {
27002702
iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND);
27012703
iwc.setIndexDeletionPolicy(combinedDeletionPolicy);
27022704
iwc.setInfoStream(TESTS_VERBOSE ? InfoStream.getDefault() : new LoggerInfoStream(logger));
2703-
iwc.setMergeScheduler(mergeScheduler);
2705+
iwc.setMergeScheduler(mergeScheduler.getMergeScheduler());
27042706
// Give us the opportunity to upgrade old segments while performing
27052707
// background merges
27062708
MergePolicy mergePolicy = config().getMergePolicy();
@@ -2753,7 +2755,7 @@ private IndexWriterConfig getIndexWriterConfig() {
27532755

27542756
/** A listener that warms the segments if needed when acquiring a new reader */
27552757
static final class RefreshWarmerListener implements BiConsumer<ElasticsearchDirectoryReader, ElasticsearchDirectoryReader> {
2756-
private final Engine.Warmer warmer;
2758+
private final Warmer warmer;
27572759
private final Logger logger;
27582760
private final AtomicBoolean isEngineClosed;
27592761

@@ -2817,6 +2819,10 @@ LiveIndexWriterConfig getCurrentIndexWriterConfig() {
28172819
return indexWriter.getConfig();
28182820
}
28192821

2822+
protected ElasticsearchMergeScheduler createMergeScheduler(ShardId shardId, IndexSettings indexSettings) {
2823+
return new EngineMergeScheduler(shardId, indexSettings);
2824+
}
2825+
28202826
private final class EngineMergeScheduler extends ElasticsearchConcurrentMergeScheduler {
28212827
private final AtomicInteger numMergesInFlight = new AtomicInteger(0);
28222828
private final AtomicBoolean isThrottling = new AtomicBoolean();
@@ -2827,7 +2833,7 @@ private final class EngineMergeScheduler extends ElasticsearchConcurrentMergeSch
28272833

28282834
@Override
28292835
public synchronized void beforeMerge(OnGoingMerge merge) {
2830-
int maxNumMerges = mergeScheduler.getMaxMergeCount();
2836+
int maxNumMerges = getMaxMergeCount();
28312837
if (numMergesInFlight.incrementAndGet() > maxNumMerges) {
28322838
if (isThrottling.getAndSet(true) == false) {
28332839
logger.info("now throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
@@ -2838,7 +2844,7 @@ public synchronized void beforeMerge(OnGoingMerge merge) {
28382844

28392845
@Override
28402846
public synchronized void afterMerge(OnGoingMerge merge) {
2841-
int maxNumMerges = mergeScheduler.getMaxMergeCount();
2847+
int maxNumMerges = getMaxMergeCount();
28422848
if (numMergesInFlight.decrementAndGet() < maxNumMerges) {
28432849
if (isThrottling.getAndSet(false)) {
28442850
logger.info("stop throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
@@ -2876,25 +2882,29 @@ protected void doRun() {
28762882

28772883
@Override
28782884
protected void handleMergeException(final Throwable exc) {
2879-
engineConfig.getThreadPool().generic().execute(new AbstractRunnable() {
2880-
@Override
2881-
public void onFailure(Exception e) {
2882-
logger.debug("merge failure action rejected", e);
2883-
}
2884-
2885-
@Override
2886-
protected void doRun() throws Exception {
2887-
/*
2888-
* We do this on another thread rather than the merge thread that we are initially called on so that we have complete
2889-
* confidence that the call stack does not contain catch statements that would cause the error that might be thrown
2890-
* here from being caught and never reaching the uncaught exception handler.
2891-
*/
2892-
failEngine("merge failed", new MergePolicy.MergeException(exc));
2893-
}
2894-
});
2885+
mergeException(exc);
28952886
}
28962887
}
28972888

2889+
protected void mergeException(final Throwable exc) {
2890+
engineConfig.getThreadPool().generic().execute(new AbstractRunnable() {
2891+
@Override
2892+
public void onFailure(Exception e) {
2893+
logger.debug("merge failure action rejected", e);
2894+
}
2895+
2896+
@Override
2897+
protected void doRun() throws Exception {
2898+
/*
2899+
* We do this on another thread rather than the merge thread that we are initially called on so that we have complete
2900+
* confidence that the call stack does not contain catch statements that would cause the error that might be thrown
2901+
* here from being caught and never reaching the uncaught exception handler.
2902+
*/
2903+
failEngine("merge failed", new MergePolicy.MergeException(exc));
2904+
}
2905+
});
2906+
}
2907+
28982908
/**
28992909
* Commits the specified index writer.
29002910
*

0 commit comments

Comments
 (0)