-
Notifications
You must be signed in to change notification settings - Fork 25k
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
Do not optimize append-only operation if normal operation with higher seq# was seen #28787
Changes from 5 commits
92671f6
ff814c8
467045a
198c504
d1ef185
3410630
c2b83f0
9856974
440f874
05050a5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -137,6 +137,7 @@ public class InternalEngine extends Engine { | |
private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false); | ||
public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; | ||
private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1); | ||
private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); | ||
private final CounterMetric numVersionLookups = new CounterMetric(); | ||
private final CounterMetric numIndexVersionsLookups = new CounterMetric(); | ||
/** | ||
|
@@ -189,7 +190,7 @@ public InternalEngine(EngineConfig engineConfig) { | |
this.combinedDeletionPolicy = new CombinedDeletionPolicy(openMode, logger, translogDeletionPolicy, | ||
translog::getLastSyncedGlobalCheckpoint, startingCommit); | ||
writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, startingCommit); | ||
updateMaxUnsafeAutoIdTimestampFromWriter(writer); | ||
bootstrapAppendOnlyInfoFromWriter(writer); | ||
assert engineConfig.getForceNewHistoryUUID() == false | ||
|| openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG | ||
|| openMode == EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG | ||
|
@@ -364,15 +365,16 @@ public int fillSeqNoGaps(long primaryTerm) throws IOException { | |
} | ||
} | ||
|
||
private void updateMaxUnsafeAutoIdTimestampFromWriter(IndexWriter writer) { | ||
long commitMaxUnsafeAutoIdTimestamp = Long.MIN_VALUE; | ||
private void bootstrapAppendOnlyInfoFromWriter(IndexWriter writer) { | ||
for (Map.Entry<String, String> entry : writer.getLiveCommitData()) { | ||
if (entry.getKey().equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) { | ||
commitMaxUnsafeAutoIdTimestamp = Long.parseLong(entry.getValue()); | ||
break; | ||
final String key = entry.getKey(); | ||
if (key.equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) { | ||
maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), Long.parseLong(entry.getValue()))); | ||
} | ||
if (key.equals(SequenceNumbers.MAX_SEQ_NO)) { | ||
maxSeqNoOfNonAppendOnlyOperations.set(Math.max(maxSeqNoOfNonAppendOnlyOperations.get(), Long.parseLong(entry.getValue()))); | ||
} | ||
} | ||
maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), commitMaxUnsafeAutoIdTimestamp)); | ||
} | ||
|
||
@Override | ||
|
@@ -893,11 +895,24 @@ public IndexResult index(Index index) throws IOException { | |
|
||
private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOException { | ||
final IndexingStrategy plan; | ||
if (canOptimizeAddDocument(index) && mayHaveBeenIndexedBefore(index) == false) { | ||
// no need to deal with out of order delivery - we never saw this one | ||
final boolean appendOnlyRequest = canOptimizeAddDocument(index); | ||
if (appendOnlyRequest && mayHaveBeenIndexedBefore(index) == false && index.seqNo() > maxSeqNoOfNonAppendOnlyOperations.get()) { | ||
/* | ||
* As soon as an append-only request was indexed into the primary, it can be exposed to a search then users can issue | ||
* a follow-up operation on it. In rare cases, the follow up operation can be arrived and processed on a replica before | ||
* the original append-only. In this case we can't simply proceed with the append only without consulting the version map. | ||
* If a replica has seen a non-append-only operation with a higher seqno than the seqno of an append-only, it may have seen | ||
* the document of that append-only request. However if the seqno of an append-only is higher than seqno of any non-append-only | ||
* requests, we can assert the replica have not seen the document of that append-only request, thus we can apply optimization. | ||
*/ | ||
assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]"; | ||
plan = IndexingStrategy.optimizedAppendOnly(index.seqNo()); | ||
} else { | ||
if (appendOnlyRequest == false) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we introduce a method similar to mayHaveBeenIndexedBefore that does the check and also updates the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Apparently @s1monw prefered the reverse. I'm fine with leaving as is. |
||
maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(index.seqNo(), curr)); | ||
assert maxSeqNoOfNonAppendOnlyOperations.get() >= index.seqNo() : "max_seqno of non-append-only was not updated;" + | ||
"max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of index [" + index.seqNo() + "]"; | ||
} | ||
versionMap.enforceSafeAccess(); | ||
// drop out of order operations | ||
assert index.versionType().versionTypeForReplicationAndRecovery() == index.versionType() : | ||
|
@@ -1032,6 +1047,11 @@ private boolean mayHaveBeenIndexedBefore(Index index) { | |
return mayHaveBeenIndexBefore; | ||
} | ||
|
||
// for testing | ||
long getMaxSeqNoOfNonAppendOnlyOperations() { | ||
return maxSeqNoOfNonAppendOnlyOperations.get(); | ||
} | ||
|
||
private static void index(final List<ParseContext.Document> docs, final IndexWriter indexWriter) throws IOException { | ||
if (docs.size() > 1) { | ||
indexWriter.addDocuments(docs); | ||
|
@@ -1187,6 +1207,9 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept | |
assert delete.versionType().versionTypeForReplicationAndRecovery() == delete.versionType() : | ||
"resolving out of order delivery based on versioning but version type isn't fit for it. got [" | ||
+ delete.versionType() + "]"; | ||
maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(delete.seqNo(), curr)); | ||
assert maxSeqNoOfNonAppendOnlyOperations.get() >= delete.seqNo() : "max_seqno of non-append-only was not updated;" + | ||
"max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of delete [" + delete.seqNo() + "]"; | ||
// unlike the primary, replicas don't really care to about found status of documents | ||
// this allows to ignore the case where a document was found in the live version maps in | ||
// a delete state and return true for the found flag in favor of code simplicity | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -4572,4 +4572,92 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup | |
} | ||
} | ||
} | ||
|
||
public void testTrackMaxSeqNoOfNonAppendOnlyOperations() throws Exception { | ||
IOUtils.close(engine, store); | ||
store = createStore(); | ||
final Path translogPath = createTempDir(); | ||
final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); | ||
try (InternalEngine engine = createEngine(store, translogPath, globalCheckpoint::get)) { | ||
final CountDownLatch latch = new CountDownLatch(1); | ||
final Thread appendOnlyIndexer = new Thread(() -> { | ||
try { | ||
latch.countDown(); | ||
final int numDocs = scaledRandomIntBetween(100, 1000); | ||
for (int i = 0; i < numDocs; i++) { | ||
ParsedDocument doc = testParsedDocument("append-only" + i, null, testDocumentWithTextField(), SOURCE, null); | ||
if (randomBoolean()) { | ||
engine.index(appendOnlyReplica(doc, randomBoolean(), 1, engine.getLocalCheckpointTracker().generateSeqNo())); | ||
} else { | ||
engine.index(appendOnlyPrimary(doc, randomBoolean(), randomNonNegativeLong())); | ||
} | ||
} | ||
} catch (Exception ex) { | ||
throw new RuntimeException("Failed to index", ex); | ||
} | ||
}); | ||
appendOnlyIndexer.setName("append-only indexer"); | ||
appendOnlyIndexer.start(); | ||
latch.await(); | ||
long maxSeqNoOfNonAppendOnly = SequenceNumbers.NO_OPS_PERFORMED; | ||
final int numOps = scaledRandomIntBetween(100, 1000); | ||
for (int i = 0; i < numOps; i++) { | ||
ParsedDocument parsedDocument = testParsedDocument(Integer.toString(i), null, testDocumentWithTextField(), SOURCE, null); | ||
if (randomBoolean()) { // On replica - update max_seqno for non-append-only operations | ||
final long seqno = engine.getLocalCheckpointTracker().generateSeqNo(); | ||
final Engine.Index doc = replicaIndexForDoc(parsedDocument, 1, seqno, randomBoolean()); | ||
if (randomBoolean()) { | ||
engine.index(doc); | ||
} else { | ||
engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), seqno, doc.primaryTerm(), | ||
doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis())); | ||
} | ||
maxSeqNoOfNonAppendOnly = seqno; | ||
} else { // On primary - do not update max_seqno for non-append-only operations | ||
if (randomBoolean()) { | ||
engine.index(indexForDoc(parsedDocument)); | ||
} else { | ||
engine.delete(new Engine.Delete(parsedDocument.type(), parsedDocument.id(), newUid(parsedDocument.id()))); | ||
} | ||
} | ||
} | ||
appendOnlyIndexer.join(120_000); | ||
assertThat(engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(maxSeqNoOfNonAppendOnly)); | ||
globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); | ||
engine.syncTranslog(); | ||
engine.flush(); | ||
} | ||
try (InternalEngine engine = createEngine(store, translogPath, globalCheckpoint::get)) { | ||
assertThat("max_seqno from non-append-only was not bootstrap from the safe commit", | ||
engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(globalCheckpoint.get())); | ||
} | ||
} | ||
|
||
public void testSkipOptimizeForExposedAppendOnlyOperations() throws Exception { | ||
long lookupTimes = 0L; | ||
final LocalCheckpointTracker localCheckpointTracker = engine.getLocalCheckpointTracker(); | ||
final int initDocs = between(0, 10); | ||
for (int i = 0; i < initDocs; i++) { | ||
index(engine, i); | ||
lookupTimes++; | ||
} | ||
// doc1 is delayed and arrived after a non-append-only op. | ||
final long seqNoDoc1 = localCheckpointTracker.generateSeqNo(); | ||
Engine.IndexResult regularDoc = engine.index(replicaIndexForDoc( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we also test a delete? |
||
testParsedDocument("d", null, testDocumentWithTextField(), SOURCE, null), 1, localCheckpointTracker.generateSeqNo(), false)); | ||
lookupTimes++; | ||
assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); // | ||
assertThat(engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(regularDoc.getSeqNo())); | ||
|
||
// should not optimize for doc1 and process as a regular doc (eg. look up in version map) | ||
engine.index(appendOnlyReplica(testParsedDocument("append-only-1", null, testDocumentWithTextField(), SOURCE, null), | ||
false, randomNonNegativeLong(), seqNoDoc1)); | ||
lookupTimes++; | ||
assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); | ||
|
||
// optimize for other append-only 2 (its seqno > max_seqno of non-append-only) - do not look up in version map. | ||
engine.index(appendOnlyReplica(testParsedDocument("append-only-2", null, testDocumentWithTextField(), SOURCE, null), | ||
false, randomNonNegativeLong(), localCheckpointTracker.generateSeqNo())); | ||
assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
question - why the leniency with max?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I removed the max expr.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like the max is still here for time stamps? maybe assert it's -1 (for both seq# and timestamp)