Skip to content

Commit 36dd4c1

Browse files
dnhatnhowardhuanghua
authored andcommitted
Avoid unneeded refresh with concurrent realtime gets (elastic#47895)
This change should reduce refreshes for a use-case where we perform multiple realtime gets at the same time on an active index. Currently, we only call refresh if the index operation is still on the versionMap. However, at the time we call refresh, that operation might be already or will be included in the latest reader. Hence, we do not need to refresh. Adding another lock here is not an issue as the refresh is already sequential.
1 parent 38ce11c commit 36dd4c1

File tree

2 files changed

+57
-1
lines changed

2 files changed

+57
-1
lines changed

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -671,7 +671,8 @@ public GetResult get(Get get, BiFunction<String, SearcherScope, Engine.Searcher>
671671
trackTranslogLocation.set(true);
672672
}
673673
}
674-
refresh("realtime_get", SearcherScope.INTERNAL, true);
674+
assert versionValue.seqNo >= 0 : versionValue;
675+
refreshIfNeeded("realtime_get", versionValue.seqNo);
675676
}
676677
scope = SearcherScope.INTERNAL;
677678
} else {

server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

Lines changed: 55 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -165,6 +165,7 @@
165165
import java.util.concurrent.BrokenBarrierException;
166166
import java.util.concurrent.CountDownLatch;
167167
import java.util.concurrent.CyclicBarrier;
168+
import java.util.concurrent.Phaser;
168169
import java.util.concurrent.Semaphore;
169170
import java.util.concurrent.TimeUnit;
170171
import java.util.concurrent.atomic.AtomicBoolean;
@@ -6102,4 +6103,58 @@ private void runTestDeleteFailure(
61026103
}
61036104
}
61046105

6106+
public void testRealtimeGetOnlyRefreshIfNeeded() throws Exception {
6107+
final AtomicInteger refreshCount = new AtomicInteger();
6108+
final ReferenceManager.RefreshListener refreshListener = new ReferenceManager.RefreshListener() {
6109+
@Override
6110+
public void beforeRefresh() {
6111+
6112+
}
6113+
6114+
@Override
6115+
public void afterRefresh(boolean didRefresh) {
6116+
if (didRefresh) {
6117+
refreshCount.incrementAndGet();
6118+
}
6119+
}
6120+
};
6121+
try (Store store = createStore()) {
6122+
final EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null,
6123+
refreshListener, null, null, engine.config().getCircuitBreakerService());
6124+
try (InternalEngine engine = createEngine(config)) {
6125+
int numDocs = randomIntBetween(10, 100);
6126+
Set<String> ids = new HashSet<>();
6127+
for (int i = 0; i < numDocs; i++) {
6128+
String id = Integer.toString(i);
6129+
engine.index(indexForDoc(createParsedDoc(id, null)));
6130+
ids.add(id);
6131+
}
6132+
final int refreshCountBeforeGet = refreshCount.get();
6133+
Thread[] getters = new Thread[randomIntBetween(1, 4)];
6134+
Phaser phaser = new Phaser(getters.length + 1);
6135+
for (int t = 0; t < getters.length; t++) {
6136+
getters[t] = new Thread(() -> {
6137+
phaser.arriveAndAwaitAdvance();
6138+
int iters = randomIntBetween(1, 10);
6139+
for (int i = 0; i < iters; i++) {
6140+
ParsedDocument doc = createParsedDoc(randomFrom(ids), null);
6141+
try (Engine.GetResult getResult = engine.get(newGet(true, doc), engine::acquireSearcher)) {
6142+
assertThat(getResult.exists(), equalTo(true));
6143+
assertThat(getResult.docIdAndVersion(), notNullValue());
6144+
}
6145+
}
6146+
});
6147+
getters[t].start();
6148+
}
6149+
phaser.arriveAndAwaitAdvance();
6150+
for (int i = 0; i < numDocs; i++) {
6151+
engine.index(indexForDoc(createParsedDoc("more-" + i, null)));
6152+
}
6153+
for (Thread getter : getters) {
6154+
getter.join();
6155+
}
6156+
assertThat(refreshCount.get(), lessThanOrEqualTo(refreshCountBeforeGet + 1));
6157+
}
6158+
}
6159+
}
61056160
}

0 commit comments

Comments
 (0)