Skip to content

Commit

Permalink
fix refresh
Browse files Browse the repository at this point in the history
  • Loading branch information
dnhatn committed Aug 29, 2018
1 parent 4c73e52 commit 4ed7907
Show file tree
Hide file tree
Showing 2 changed files with 42 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1420,6 +1420,7 @@ final void refresh(String source, SearcherScope scope) throws EngineException {
// since it flushes the index as well (though, in terms of concurrency, we are allowed to do it)
// both refresh types will result in an internal refresh but only the external will also
// pass the new reader reference to the external reader manager.
final long localCheckpointBeforeRefresh = getLocalCheckpoint();

// this will also cause version map ram to be freed hence we always account for it.
final long bytes = indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh();
Expand All @@ -1445,6 +1446,7 @@ final void refresh(String source, SearcherScope scope) throws EngineException {
} finally {
store.decRef();
}
lastRefreshedCheckpointListener.updateRefreshedCheckpoint(localCheckpointBeforeRefresh);
}
} catch (AlreadyClosedException e) {
failOnTragicEvent(e);
Expand All @@ -1459,7 +1461,8 @@ final void refresh(String source, SearcherScope scope) throws EngineException {
} finally {
writingBytes.addAndGet(-bytes);
}

assert lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; " +
"local_checkpoint=" + localCheckpointBeforeRefresh + " refresh_checkpoint=" + lastRefreshedCheckpoint();
// TODO: maybe we should just put a scheduled job in threadPool?
// We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes
// for a long time:
Expand Down Expand Up @@ -2526,21 +2529,31 @@ public long softUpdateDocuments(Term term, Iterable<? extends Iterable<? extends
final long lastRefreshedCheckpoint() {
return lastRefreshedCheckpointListener.refreshedCheckpoint.get();
}

private final class LastRefreshedCheckpointListener implements ReferenceManager.RefreshListener {
final AtomicLong refreshedCheckpoint;
private long pendingCheckpoint;

LastRefreshedCheckpointListener(long initialLocalCheckpoint) {
this.refreshedCheckpoint = new AtomicLong(initialLocalCheckpoint);
}

@Override
public void beforeRefresh() {
pendingCheckpoint = localCheckpointTracker.getCheckpoint(); // All change until this point should be visible after refresh
// all changes until this point should be visible after refresh
pendingCheckpoint = localCheckpointTracker.getCheckpoint();
}

@Override
public void afterRefresh(boolean didRefresh) {
if (didRefresh) {
refreshedCheckpoint.set(pendingCheckpoint);
updateRefreshedCheckpoint(pendingCheckpoint);
}
}

void updateRefreshedCheckpoint(long checkpoint) {
refreshedCheckpoint.updateAndGet(curr -> Math.max(curr, checkpoint));
assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -4994,6 +4994,32 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException {
}
}

public void testLastRefreshCheckpoint() throws Exception {
AtomicBoolean done = new AtomicBoolean();
Thread[] refreshThreads = new Thread[between(1, 8)];
CountDownLatch latch = new CountDownLatch(refreshThreads.length);
for (int i = 0; i < refreshThreads.length; i++) {
latch.countDown();
refreshThreads[i] = new Thread(() -> {
while (done.get() == false) {
long checkPointBeforeRefresh = engine.getLocalCheckpoint();
engine.refresh("test", randomFrom(Engine.SearcherScope.values()));
assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh));
}
});
refreshThreads[i].start();
}
latch.await();
List<Engine.Operation> ops = generateSingleDocHistory(true, VersionType.EXTERNAL, 1, 10, 1000, "1");
concurrentlyApplyOps(ops, engine);
done.set(true);
for (Thread thread : refreshThreads) {
thread.join();
}
engine.refresh("test");
assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint()));
}

private static void trimUnsafeCommits(EngineConfig config) throws IOException {
final Store store = config.getStore();
final TranslogConfig translogConfig = config.getTranslogConfig();
Expand Down

0 comments on commit 4ed7907

Please sign in to comment.