Skip to content
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

Avoid unneeded refresh with concurrent realtime gets #47895

Merged
merged 4 commits into from
Oct 13, 2019
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -671,7 +671,7 @@ public GetResult get(Get get, BiFunction<String, SearcherScope, Engine.Searcher>
trackTranslogLocation.set(true);
}
}
refresh("realtime_get", SearcherScope.INTERNAL, true);
refreshIfNeeded("realtime_get", versionValue.seqNo);
dnhatn marked this conversation as resolved.
Show resolved Hide resolved
}
scope = SearcherScope.INTERNAL;
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,7 @@
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Phaser;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
Expand Down Expand Up @@ -6102,4 +6103,57 @@ private void runTestDeleteFailure(
}
}

public void testRealtimeGetOnlyRefreshIfNeeded() throws Exception {
final AtomicInteger refreshCount = new AtomicInteger();
final ReferenceManager.RefreshListener refreshListener = new ReferenceManager.RefreshListener() {
@Override
public void beforeRefresh() {

}
@Override
dnhatn marked this conversation as resolved.
Show resolved Hide resolved
public void afterRefresh(boolean didRefresh) {
if (didRefresh) {
refreshCount.incrementAndGet();
}
}
};
try (Store store = createStore()) {
final EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null,
refreshListener, null, null, engine.config().getCircuitBreakerService());
try (InternalEngine engine = createEngine(config)) {
int numDocs = randomIntBetween(10, 100);
Set<String> ids = new HashSet<>();
for (int i = 0; i < numDocs; i++) {
String id = Integer.toString(i);
engine.index(indexForDoc(createParsedDoc(id, null)));
ids.add(id);
}
final int refreshCountBeforeGet = refreshCount.get();
Thread[] getters = new Thread[randomIntBetween(1, 4)];
Phaser phaser = new Phaser(getters.length + 1);
for (int t = 0; t < getters.length; t++) {
getters[t] = new Thread(() -> {
phaser.arriveAndAwaitAdvance();
int iters = randomIntBetween(1, 10);
for (int i = 0; i < iters; i++) {
ParsedDocument doc = createParsedDoc(randomFrom(ids), null);
try (Engine.GetResult getResult = engine.get(newGet(true, doc), engine::acquireSearcher)) {
assertThat(getResult.exists(), equalTo(true));
assertThat(getResult.docIdAndVersion(), notNullValue());
}
}
});
getters[t].start();
}
phaser.arriveAndAwaitAdvance();
for (int i = 0; i < numDocs; i++) {
engine.index(indexForDoc(createParsedDoc("more-" + i, null)));
}
for (Thread getter : getters) {
getter.join();
}
assertThat(refreshCount.get(), lessThanOrEqualTo(refreshCountBeforeGet + 1));
}
}
}
}