Skip to content

Commit

Permalink
Prune only gc deletes below local checkpoint (#28790)
Browse files Browse the repository at this point in the history
Once a document is deleted and Lucene is refreshed, we will not be able 
to look up the `version/seq#` associated with that delete in Lucene. As
conflicting operations can still be indexed, we need another mechanism
to remember these deletes. Therefore deletes should still be stored in
the Version Map, even after Lucene is refreshed. Obviously, we can't
remember all deletes forever so a trimming mechanism is needed.
Currently, we remember deletes for at least 1 minute (the default GC
deletes cycle) and clean them periodically. This is, at the moment, the
best we can do on the primary for user facing APIs but this arbitrary
time limit is problematic for replicas. Furthermore, we can't rely on
the primary and replicas doing the trimming in a synchronized manner,
and failing to do so results in the replica and primary making different
decisions. 

The following scenario can cause inconsistency between
primary and replica.

1. Primary index doc (index, id=1, v2)
2. Network packet issue causes index operation to back off and wait
3. Primary deletes doc (delete, id=1, v3)
4. Replica processes delete (delete, id=1, v3)
5. 1+ minute passes (GC deletes runs replica)
6. Indexing op is finally sent to the replica which no processes it 
   because it forgot about the delete.

We can reply on sequence-numbers to prevent this issue. If we prune only 
deletes whose seqno at most the local checkpoint, a replica will
correctly remember what it needs. The correctness is explained as
follows:

Suppose o1 and o2 are two operations on the same document with seq#(o1) 
< seq#(o2), and o2 arrives before o1 on the replica. o2 is processed
normally since it arrives first; when o1 arrives it should be discarded:
 
1. If seq#(o1) <= LCP, then it will be not be added to Lucene, as it was
  already previously added.

2. If seq#(o1)  > LCP, then it depends on the nature of o2:
  - If o2 is a delete then its seq# is recorded in the VersionMap,
    since seq#(o2) > seq#(o1) > LCP, so a lookup can find it and
    determine that o1 is stale.
  
  - If o2 is an indexing then its seq# is either in Lucene (if
    refreshed) or the VersionMap (if not refreshed yet), so a 
    real-time lookup can find it and determine that o1 is stale.

In this PR, we prefer to deploy a single trimming strategy, which 
satisfies both requirements, on primary and replicas because:

- It's simpler - no need to distinguish if an engine is running at
primary mode or replica mode or being promoted.

- If a replica subsequently is promoted, user experience is fully
maintained as that replica remembers deletes for the last GC cycle.

However, the version map may consume less memory if we deploy two 
different trimming strategies for primary and replicas.
  • Loading branch information
dnhatn authored Mar 26, 2018
1 parent bca2646 commit 8795760
Show file tree
Hide file tree
Showing 7 changed files with 211 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -1540,15 +1541,41 @@ public void trimTranslog() throws EngineException {
}

private void pruneDeletedTombstones() {
/*
* We need to deploy two different trimming strategies for GC deletes on primary and replicas. Delete operations on primary
* are remembered for at least one GC delete cycle and trimmed periodically. This is, at the moment, the best we can do on
* primary for user facing APIs but this arbitrary time limit is problematic for replicas. On replicas however we should
* trim only deletes whose seqno at most the local checkpoint. This requirement is explained as follows.
*
* Suppose o1 and o2 are two operations on the same document with seq#(o1) < seq#(o2), and o2 arrives before o1 on the replica.
* o2 is processed normally since it arrives first; when o1 arrives it should be discarded:
* - If seq#(o1) <= LCP, then it will be not be added to Lucene, as it was already previously added.
* - If seq#(o1) > LCP, then it depends on the nature of o2:
* *) If o2 is a delete then its seq# is recorded in the VersionMap, since seq#(o2) > seq#(o1) > LCP,
* so a lookup can find it and determine that o1 is stale.
* *) If o2 is an indexing then its seq# is either in Lucene (if refreshed) or the VersionMap (if not refreshed yet),
* so a real-time lookup can find it and determine that o1 is stale.
*
* Here we prefer to deploy a single trimming strategy, which satisfies two constraints, on both primary and replicas because:
* - It's simpler - no need to distinguish if an engine is running at primary mode or replica mode or being promoted.
* - If a replica subsequently is promoted, user experience is maintained as that replica remembers deletes for the last GC cycle.
*
* However, the version map may consume less memory if we deploy two different trimming strategies for primary and replicas.
*/
final long timeMSec = engineConfig.getThreadPool().relativeTimeInMillis();
versionMap.pruneTombstones(timeMSec, engineConfig.getIndexSettings().getGcDeletesInMillis());
final long maxTimestampToPrune = timeMSec - engineConfig.getIndexSettings().getGcDeletesInMillis();
versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getCheckpoint());
lastDeleteVersionPruneTimeMSec = timeMSec;
}

// testing
void clearDeletedTombstones() {
// clean with current time Long.MAX_VALUE and interval 0 since we use a greater than relationship here.
versionMap.pruneTombstones(Long.MAX_VALUE, 0);
versionMap.pruneTombstones(Long.MAX_VALUE, localCheckpointTracker.getMaxSeqNo());
}

// for testing
final Collection<DeleteVersionValue> getDeletedTombstones() {
return versionMap.getAllTombstones().values();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -375,21 +375,25 @@ void removeTombstoneUnderLock(BytesRef uid) {
}
}

private boolean canRemoveTombstone(long currentTime, long pruneInterval, DeleteVersionValue versionValue) {
// check if the value is old enough to be removed
final boolean isTooOld = currentTime - versionValue.time > pruneInterval;
private boolean canRemoveTombstone(long maxTimestampToPrune, long maxSeqNoToPrune, DeleteVersionValue versionValue) {
// check if the value is old enough and safe to be removed
final boolean isTooOld = versionValue.time < maxTimestampToPrune;
final boolean isSafeToPrune = versionValue.seqNo <= maxSeqNoToPrune;
// version value can't be removed it's
// not yet flushed to lucene ie. it's part of this current maps object
final boolean isNotTrackedByCurrentMaps = versionValue.time < maps.getMinDeleteTimestamp();
return isTooOld && isNotTrackedByCurrentMaps;
return isTooOld && isSafeToPrune && isNotTrackedByCurrentMaps;
}

void pruneTombstones(long currentTime, long pruneInterval) {
/**
* Try to prune tombstones whose timestamp is less than maxTimestampToPrune and seqno at most the maxSeqNoToPrune.
*/
void pruneTombstones(long maxTimestampToPrune, long maxSeqNoToPrune) {
for (Map.Entry<BytesRef, DeleteVersionValue> entry : tombstones.entrySet()) {
// we do check before we actually lock the key - this way we don't need to acquire the lock for tombstones that are not
// prune-able. If the tombstone changes concurrently we will re-read and step out below since if we can't collect it now w
// we won't collect the tombstone below since it must be newer than this one.
if (canRemoveTombstone(currentTime, pruneInterval, entry.getValue())) {
if (canRemoveTombstone(maxTimestampToPrune, maxSeqNoToPrune, entry.getValue())) {
final BytesRef uid = entry.getKey();
try (Releasable lock = keyedLock.tryAcquire(uid)) {
// we use tryAcquire here since this is a best effort and we try to be least disruptive
Expand All @@ -399,7 +403,7 @@ void pruneTombstones(long currentTime, long pruneInterval) {
// Must re-get it here, vs using entry.getValue(), in case the uid was indexed/deleted since we pulled the iterator:
final DeleteVersionValue versionValue = tombstones.get(uid);
if (versionValue != null) {
if (canRemoveTombstone(currentTime, pruneInterval, versionValue)) {
if (canRemoveTombstone(maxTimestampToPrune, maxSeqNoToPrune, versionValue)) {
removeTombstoneUnderLock(uid);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
Expand Down Expand Up @@ -163,6 +164,8 @@
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.sameInstance;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.everyItem;
import static org.hamcrest.Matchers.greaterThan;
Expand All @@ -173,6 +176,8 @@
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;

public class InternalEngineTests extends EngineTestCase {

Expand Down Expand Up @@ -4464,4 +4469,65 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup
}
}
}

public void testPruneOnlyDeletesAtMostLocalCheckpoint() throws Exception {
final AtomicLong clock = new AtomicLong(0);
threadPool = spy(threadPool);
when(threadPool.relativeTimeInMillis()).thenAnswer(invocation -> clock.get());
final EngineConfig config = engine.config();
final long gcInterval = randomIntBetween(0, 10);
final IndexSettings indexSettings = engine.config().getIndexSettings();
final IndexMetaData indexMetaData = IndexMetaData.builder(indexSettings.getIndexMetaData())
.settings(Settings.builder().put(indexSettings.getSettings())
.put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), TimeValue.timeValueMillis(gcInterval).getStringRep())).build();
indexSettings.updateIndexMetaData(indexMetaData);

try (Store store = createStore();
InternalEngine engine = createEngine(new EngineConfig(config.getShardId(), config.getAllocationId(), threadPool,
indexSettings, config.getWarmer(), store, config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(),
new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(),
config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(),
config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(),
config.getGlobalCheckpointSupplier()))) {
engine.config().setEnableGcDeletes(false);
for (int i = 0, docs = scaledRandomIntBetween(0, 10); i < docs; i++) {
index(engine, i);
}
final long deleteBatch = between(10, 20);
final long gapSeqNo = randomLongBetween(
engine.getLocalCheckpointTracker().getMaxSeqNo() + 1, engine.getLocalCheckpointTracker().getMaxSeqNo() + deleteBatch);
for (int i = 0; i < deleteBatch; i++) {
final long seqno = engine.getLocalCheckpointTracker().generateSeqNo();
if (seqno != gapSeqNo) {
if (randomBoolean()) {
clock.incrementAndGet();
}
engine.delete(replicaDeleteForDoc(UUIDs.randomBase64UUID(), 1, seqno, threadPool.relativeTimeInMillis()));
}
}
List<DeleteVersionValue> tombstones = new ArrayList<>(engine.getDeletedTombstones());
engine.config().setEnableGcDeletes(true);
// Prune tombstones whose seqno < gap_seqno and timestamp < clock-gcInterval.
clock.set(randomLongBetween(gcInterval, deleteBatch + gcInterval));
engine.refresh("test");
tombstones.removeIf(v -> v.seqNo < gapSeqNo && v.time < clock.get() - gcInterval);
assertThat(engine.getDeletedTombstones(), containsInAnyOrder(tombstones.toArray()));
// Prune tombstones whose seqno at most the local checkpoint (eg. seqno < gap_seqno).
clock.set(randomLongBetween(deleteBatch + gcInterval * 4/3, 100)); // Need a margin for gcInterval/4.
engine.refresh("test");
tombstones.removeIf(v -> v.seqNo < gapSeqNo);
assertThat(engine.getDeletedTombstones(), containsInAnyOrder(tombstones.toArray()));
// Fill the seqno gap - should prune all tombstones.
clock.set(between(0, 100));
if (randomBoolean()) {
engine.index(replicaIndexForDoc(testParsedDocument("d", null, testDocumentWithTextField(), SOURCE, null), 1, gapSeqNo, false));
} else {
engine.delete(replicaDeleteForDoc(UUIDs.randomBase64UUID(), Versions.MATCH_ANY, gapSeqNo, threadPool.relativeTimeInMillis()));
}
clock.set(randomLongBetween(100 + gcInterval * 4/3, Long.MAX_VALUE)); // Need a margin for gcInterval/4.
engine.refresh("test");
assertThat(engine.getDeletedTombstones(), empty());
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.StreamSupport;

import static org.hamcrest.Matchers.empty;

public class LiveVersionMapTests extends ESTestCase {

Expand Down Expand Up @@ -106,14 +107,15 @@ public void testBasics() throws IOException {
map.afterRefresh(randomBoolean());
assertNull(map.getUnderLock(uid("test")));


map.putUnderLock(uid("test"), new DeleteVersionValue(1,1,1,1));
assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test")));
map.beforeRefresh();
assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test")));
map.afterRefresh(randomBoolean());
assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test")));
map.pruneTombstones(2, 0);
assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test")));
map.pruneTombstones(2, 1);
assertNull(map.getUnderLock(uid("test")));
}
}
Expand All @@ -134,8 +136,10 @@ public void testConcurrently() throws IOException, InterruptedException {
CountDownLatch startGun = new CountDownLatch(numThreads);
CountDownLatch done = new CountDownLatch(numThreads);
int randomValuesPerThread = randomIntBetween(5000, 20000);
AtomicLong clock = new AtomicLong(0);
AtomicLong lastPrunedTimestamp = new AtomicLong(-1);
final AtomicLong clock = new AtomicLong(0);
final AtomicLong lastPrunedTimestamp = new AtomicLong(-1);
final AtomicLong maxSeqNo = new AtomicLong();
final AtomicLong lastPrunedSeqNo = new AtomicLong();
for (int j = 0; j < threads.length; j++) {
threads[j] = new Thread(() -> {
startGun.countDown();
Expand All @@ -148,29 +152,31 @@ public void testConcurrently() throws IOException, InterruptedException {
try {
for (int i = 0; i < randomValuesPerThread; ++i) {
BytesRef bytesRef = randomFrom(random(), keyList);
final long clockTick = clock.get();
try (Releasable r = map.acquireLock(bytesRef)) {
VersionValue versionValue = values.computeIfAbsent(bytesRef,
v -> new VersionValue(randomLong(), randomLong(), randomLong()));
v -> new VersionValue(randomLong(), maxSeqNo.incrementAndGet(), randomLong()));
boolean isDelete = versionValue instanceof DeleteVersionValue;
if (isDelete) {
map.removeTombstoneUnderLock(bytesRef);
deletes.remove(bytesRef);
}
if (isDelete == false && rarely()) {
versionValue = new DeleteVersionValue(versionValue.version + 1, versionValue.seqNo + 1,
versionValue = new DeleteVersionValue(versionValue.version + 1, maxSeqNo.incrementAndGet(),
versionValue.term, clock.getAndIncrement());
deletes.put(bytesRef, (DeleteVersionValue) versionValue);
} else {
versionValue = new VersionValue(versionValue.version + 1, versionValue.seqNo + 1, versionValue.term);
versionValue = new VersionValue(versionValue.version + 1, maxSeqNo.incrementAndGet(), versionValue.term);
}
values.put(bytesRef, versionValue);
map.putUnderLock(bytesRef, versionValue);
}
if (rarely()) {
map.pruneTombstones(clockTick, 0);
// timestamp we pruned the deletes
lastPrunedTimestamp.updateAndGet(prev -> Math.max(clockTick, prev)); // make sure we track the latest
final long pruneSeqNo = randomLongBetween(0, maxSeqNo.get());
final long clockTick = randomLongBetween(0, clock.get());
map.pruneTombstones(clockTick, pruneSeqNo);
// make sure we track the latest timestamp and seqno we pruned the deletes
lastPrunedTimestamp.updateAndGet(prev -> Math.max(clockTick, prev));
lastPrunedSeqNo.updateAndGet(prev -> Math.max(pruneSeqNo, prev));
}
}
} finally {
Expand Down Expand Up @@ -234,15 +240,17 @@ public void testConcurrently() throws IOException, InterruptedException {
VersionValue value = map.getUnderLock(e.getKey());
// here we keep track of the deletes and ensure that all deletes that are not visible anymore ie. not in the map
// have a timestamp that is smaller or equal to the maximum timestamp that we pruned on
final DeleteVersionValue delete = e.getValue();
if (value == null) {
assertTrue(e.getValue().time + " > " + lastPrunedTimestamp.get(), e.getValue().time <= lastPrunedTimestamp.get());
assertTrue(delete.time + " > " + lastPrunedTimestamp.get() + "," + delete.seqNo + " > " + lastPrunedSeqNo.get(),
delete.time <= lastPrunedTimestamp.get() && delete.seqNo <= lastPrunedSeqNo.get());
} else {
assertEquals(value, e.getValue());
assertEquals(value, delete);
}
}
});
map.pruneTombstones(clock.incrementAndGet(), 0);
assertEquals(0, StreamSupport.stream(map.getAllTombstones().entrySet().spliterator(), false).count());
map.pruneTombstones(clock.incrementAndGet(), maxSeqNo.get());
assertThat(map.getAllTombstones().entrySet(), empty());
}

public void testCarryOnSafeAccess() throws IOException {
Expand Down
Loading

0 comments on commit 8795760

Please sign in to comment.