Optimize version map for append-only indexing (#27752)

Today we still maintain a version map even if we only index append-only
or in other words, documents with auto-generated IDs. We can instead maintain
an un-safe version map that will be swapped to a safe version map only if necessary
once we see the first document that requires access to the version map. For instance:
 * a auto-generated id retry
 * any kind of deletes
 * a document with a foreign ID (non-autogenerated

In these cases we forcefully refresh then internal reader and start maintaining
a version map until such a safe map wasn't necessary for two refresh cycles.
Indices / shards that never see an autogenerated ID document will always meintain a version
map and in the case of a delete / retry in a pure append-only index the version map will be
de-optimized for a short amount of time until we know it's safe again to swap back. This
will also minimize the requried refeshes.

Closes #19813
This commit is contained in:
Simon Willnauer 2017-12-15 12:13:10 +01:00 committed by GitHub
parent cef7bd2079
commit d941c64edb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 351 additions and 20 deletions

View File

@ -562,7 +562,7 @@ public class InternalEngine extends Engine {
ensureOpen();
SearcherScope scope;
if (get.realtime()) {
VersionValue versionValue = versionMap.getUnderLock(get.uid().bytes());
VersionValue versionValue = getVersionFromMap(get.uid().bytes());
if (versionValue != null) {
if (versionValue.isDelete()) {
return GetResult.NOT_EXISTS;
@ -600,7 +600,7 @@ public class InternalEngine extends Engine {
private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) throws IOException {
assert op.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO : "resolving ops based on seq# but no seqNo is found";
final OpVsLuceneDocStatus status;
final VersionValue versionValue = versionMap.getUnderLock(op.uid().bytes());
VersionValue versionValue = getVersionFromMap(op.uid().bytes());
assert incrementVersionLookup();
if (versionValue != null) {
if (op.seqNo() > versionValue.seqNo ||
@ -637,7 +637,7 @@ public class InternalEngine extends Engine {
/** resolves the current version of the document, returning null if not found */
private VersionValue resolveDocVersion(final Operation op) throws IOException {
assert incrementVersionLookup(); // used for asserting in tests
VersionValue versionValue = versionMap.getUnderLock(op.uid().bytes());
VersionValue versionValue = getVersionFromMap(op.uid().bytes());
if (versionValue == null) {
assert incrementIndexVersionLookup(); // used for asserting in tests
final long currentVersion = loadCurrentVersionFromIndex(op.uid());
@ -651,6 +651,21 @@ public class InternalEngine extends Engine {
return versionValue;
}
private VersionValue getVersionFromMap(BytesRef id) {
if (versionMap.isUnsafe()) {
synchronized (versionMap) {
// we are switching from an unsafe map to a safe map. This might happen concurrently
// but we only need to do this once since the last operation per ID is to add to the version
// map so once we pass this point we can safely lookup from the version map.
if (versionMap.isUnsafe()) {
refresh("unsafe_version_map", SearcherScope.INTERNAL);
}
versionMap.enforceSafeAccess();
}
}
return versionMap.getUnderLock(id);
}
private boolean canOptimizeAddDocument(Index index) {
if (index.getAutoGeneratedIdTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) {
assert index.getAutoGeneratedIdTimestamp() >= 0 : "autoGeneratedIdTimestamp must be positive but was: "
@ -812,6 +827,7 @@ public class InternalEngine extends Engine {
assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]";
plan = IndexingStrategy.optimizedAppendOnly(index.seqNo());
} else {
versionMap.enforceSafeAccess();
// drop out of order operations
assert index.versionType().versionTypeForReplicationAndRecovery() == index.versionType() :
"resolving out of order delivery based on versioning but version type isn't fit for it. got [" + index.versionType() + "]";
@ -849,10 +865,12 @@ public class InternalEngine extends Engine {
if (canOptimizeAddDocument(index)) {
if (mayHaveBeenIndexedBefore(index)) {
plan = IndexingStrategy.overrideExistingAsIfNotThere(generateSeqNoForOperation(index), 1L);
versionMap.enforceSafeAccess();
} else {
plan = IndexingStrategy.optimizedAppendOnly(generateSeqNoForOperation(index));
}
} else {
versionMap.enforceSafeAccess();
// resolves incoming version
final VersionValue versionValue = resolveDocVersion(index);
final long currentVersion;
@ -898,7 +916,7 @@ public class InternalEngine extends Engine {
assert assertDocDoesNotExist(index, canOptimizeAddDocument(index) == false);
index(index.docs(), indexWriter);
}
versionMap.putUnderLock(index.uid().bytes(),
versionMap.maybePutUnderLock(index.uid().bytes(),
new VersionValue(plan.versionForIndexing, plan.seqNoForIndexing, index.primaryTerm()));
return new IndexResult(plan.versionForIndexing, plan.seqNoForIndexing, plan.currentNotFoundOrDeleted);
} catch (Exception ex) {
@ -1018,7 +1036,9 @@ public class InternalEngine extends Engine {
* Asserts that the doc in the index operation really doesn't exist
*/
private boolean assertDocDoesNotExist(final Index index, final boolean allowDeleted) throws IOException {
final VersionValue versionValue = versionMap.getUnderLock(index.uid().bytes());
// NOTE this uses direct access to the version map since we are in the assertion code where we maintain a secondary
// map in the version map such that we don't need to refresh if we are unsafe;
final VersionValue versionValue = versionMap.getVersionForAssert(index.uid().bytes());
if (versionValue != null) {
if (versionValue.isDelete() == false || allowDeleted == false) {
throw new AssertionError("doc [" + index.type() + "][" + index.id() + "] exists in version map (version " + versionValue + ")");
@ -1044,6 +1064,7 @@ public class InternalEngine extends Engine {
@Override
public DeleteResult delete(Delete delete) throws IOException {
versionMap.enforceSafeAccess();
assert Objects.equals(delete.uid().field(), uidField) : delete.uid().field();
assert assertVersionType(delete);
assert assertIncomingSequenceNumber(delete.origin(), delete.seqNo());
@ -2114,6 +2135,15 @@ public class InternalEngine extends Engine {
return true;
}
int getVersionMapSize() {
return versionMap.getAllCurrent().size();
}
boolean isSafeAccessRequired() {
return versionMap.isSafeAccessRequired();
}
/**
* Returns <code>true</code> iff the index writer has any deletions either buffered in memory or
* in the index.

View File

@ -32,7 +32,7 @@ import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
/** Maps _uid value to its version information. */
class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
final class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
/**
* Resets the internal map and adjusts it's capacity as if there were no indexing operations.
@ -46,22 +46,100 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
maps = new Maps();
}
private static class Maps {
private static final class VersionLookup {
private static final VersionLookup EMPTY = new VersionLookup(Collections.emptyMap());
private final Map<BytesRef,VersionValue> map;
// each version map has a notion of safe / unsafe which allows us to apply certain optimization in the auto-generated ID usecase
// where we know that documents can't have any duplicates so we can skip the version map entirely. This reduces
// the memory pressure significantly for this use-case where we often get a massive amount of small document (metrics).
// if the version map is in safeAccess mode we track all version in the version map. yet if a document comes in that needs
// safe access but we are not in this mode we force a refresh and make the map as safe access required. All subsequent ops will
// respect that and fill the version map. The nice part here is that we are only really requiring this for a single ID and since
// we hold the ID lock in the engine while we do all this it's safe to do it globally unlocked.
// NOTE: these values can both be non-volatile since it's ok to read a stale value per doc ID. We serialize changes in the engine
// that will prevent concurrent updates to the same document ID and therefore we can rely on the happens-before guanratee of the
// map reference itself.
private boolean unsafe;
private VersionLookup(Map<BytesRef, VersionValue> map) {
this.map = map;
}
VersionValue get(BytesRef key) {
return map.get(key);
}
VersionValue put(BytesRef key, VersionValue value) {
return map.put(key, value);
}
boolean isEmpty() {
return map.isEmpty();
}
int size() {
return map.size();
}
boolean isUnsafe() {
return unsafe;
}
void markAsUnsafe() {
unsafe = true;
}
}
private static final class Maps {
// All writes (adds and deletes) go into here:
final Map<BytesRef,VersionValue> current;
final VersionLookup current;
// Used while refresh is running, and to hold adds/deletes until refresh finishes. We read from both current and old on lookup:
final Map<BytesRef,VersionValue> old;
final VersionLookup old;
Maps(Map<BytesRef,VersionValue> current, Map<BytesRef,VersionValue> old) {
this.current = current;
this.old = old;
// this is not volatile since we don't need to maintain a happens before relation ship across doc IDs so it's enough to
// have the volatile read of the Maps reference to make it visible even across threads.
boolean needsSafeAccess;
final boolean previousMapsNeededSafeAccess;
Maps(VersionLookup current, VersionLookup old, boolean previousMapsNeededSafeAccess) {
this.current = current;
this.old = old;
this.previousMapsNeededSafeAccess = previousMapsNeededSafeAccess;
}
Maps() {
this(ConcurrentCollections.<BytesRef,VersionValue>newConcurrentMapWithAggressiveConcurrency(),
Collections.emptyMap());
this(new VersionLookup(ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency()), VersionLookup.EMPTY, false);
}
boolean isSafeAccessMode() {
return needsSafeAccess || previousMapsNeededSafeAccess;
}
boolean shouldInheritSafeAccess() {
final boolean mapHasNotSeenAnyOperations = current.isEmpty() && current.isUnsafe() == false;
return needsSafeAccess
// we haven't seen any ops and map before needed it so we maintain it
|| (mapHasNotSeenAnyOperations && previousMapsNeededSafeAccess);
}
/**
* Builds a new map for the refresh transition this should be called in beforeRefresh()
*/
Maps buildTransitionMap() {
return new Maps(new VersionLookup(ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(current.size())),
current, shouldInheritSafeAccess());
}
/**
* builds a new map that invalidates the old map but maintains the current. This should be called in afterRefresh()
*/
Maps invalidateOldMap() {
return new Maps(current, VersionLookup.EMPTY, previousMapsNeededSafeAccess);
}
}
@ -69,6 +147,9 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
private final Map<BytesRef,DeleteVersionValue> tombstones = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
private volatile Maps maps = new Maps();
// we maintain a second map that only receives the updates that we skip on the actual map (unsafe ops)
// this map is only maintained if assertions are enabled
private volatile Maps unsafeKeysMap = new Maps();
/** Bytes consumed for each BytesRef UID:
* In this base value, we account for the {@link BytesRef} object itself as
@ -113,8 +194,8 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
// map. While reopen is running, any lookup will first
// try this new map, then fallback to old, then to the
// current searcher:
maps = new Maps(ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(maps.current.size()), maps.current);
maps = maps.buildTransitionMap();
assert (unsafeKeysMap = unsafeKeysMap.buildTransitionMap()) != null;
// This is not 100% correct, since concurrent indexing ops can change these counters in between our execution of the previous
// line and this one, but that should be minor, and the error won't accumulate over time:
ramBytesUsedCurrent.set(0);
@ -128,13 +209,18 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
// case. This is because we assign new maps (in beforeRefresh) slightly before Lucene actually flushes any segments for the
// reopen, and so any concurrent indexing requests can still sneak in a few additions to that current map that are in fact reflected
// in the previous reader. We don't touch tombstones here: they expire on their own index.gc_deletes timeframe:
maps = new Maps(maps.current, Collections.emptyMap());
maps = maps.invalidateOldMap();
assert (unsafeKeysMap = unsafeKeysMap.invalidateOldMap()) != null;
}
/** Returns the live version (add or delete) for this uid. */
VersionValue getUnderLock(final BytesRef uid) {
Maps currentMaps = maps;
return getUnderLock(uid, maps);
}
private VersionValue getUnderLock(final BytesRef uid, Maps currentMaps) {
// First try to get the "live" value:
VersionValue value = currentMaps.current.get(uid);
if (value != null) {
@ -149,11 +235,52 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
return tombstones.get(uid);
}
VersionValue getVersionForAssert(final BytesRef uid) {
VersionValue value = getUnderLock(uid, maps);
if (value == null) {
value = getUnderLock(uid, unsafeKeysMap);
}
return value;
}
boolean isUnsafe() {
return maps.current.isUnsafe() || maps.old.isUnsafe();
}
void enforceSafeAccess() {
maps.needsSafeAccess = true;
}
boolean isSafeAccessRequired() {
return maps.isSafeAccessMode();
}
/** Adds this uid/version to the pending adds map iff the map needs safe access. */
void maybePutUnderLock(BytesRef uid, VersionValue version) {
Maps maps = this.maps;
if (maps.isSafeAccessMode()) {
putUnderLock(uid, version, maps);
} else {
maps.current.markAsUnsafe();
assert putAssertionMap(uid, version);
}
}
private boolean putAssertionMap(BytesRef uid, VersionValue version) {
putUnderLock(uid, version, unsafeKeysMap);
return true;
}
/** Adds this uid/version to the pending adds map. */
void putUnderLock(BytesRef uid, VersionValue version) {
Maps maps = this.maps;
putUnderLock(uid, version, maps);
}
/** Adds this uid/version to the pending adds map. */
private void putUnderLock(BytesRef uid, VersionValue version, Maps maps) {
assert uid.bytes.length == uid.length : "Oversized _uid! UID length: " + uid.length + ", bytes length: " + uid.bytes.length;
long uidRAMBytesUsed = BASE_BYTES_PER_BYTESREF + uid.bytes.length;
final VersionValue prev = maps.current.put(uid, version);
if (prev != null) {
// Deduct RAM for the version we just replaced:
@ -264,5 +391,5 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
/** Returns the current internal versions as a point in time snapshot*/
Map<BytesRef, VersionValue> getAllCurrent() {
return maps.current;
return maps.current.map;
}}

View File

@ -106,6 +106,7 @@ import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.RootObjectMapper;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.seqno.SeqNoStats;
import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.seqno.SequenceNumbersService;
@ -175,6 +176,69 @@ import static org.hamcrest.Matchers.nullValue;
public class InternalEngineTests extends EngineTestCase {
public void testVersionMapAfterAutoIDDocument() throws IOException {
ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField("test"),
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
Engine.Index operation = randomBoolean() ?
appendOnlyPrimary(doc, false, 1)
: appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5));
engine.index(operation);
assertFalse(engine.isSafeAccessRequired());
doc = testParsedDocument("1", null, testDocumentWithTextField("updated"),
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
Engine.Index update = indexForDoc(doc);
engine.index(update);
assertTrue(engine.isSafeAccessRequired());
assertEquals(1, engine.getVersionMapSize());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
assertEquals(0, searcher.reader().numDocs());
}
try (Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL)) {
assertEquals(1, searcher.reader().numDocs());
TopDocs search = searcher.searcher().search(new MatchAllDocsQuery(), 1);
org.apache.lucene.document.Document luceneDoc = searcher.searcher().doc(search.scoreDocs[0].doc);
assertEquals("test", luceneDoc.get("value"));
}
// now lets make this document visible
engine.refresh("test");
if (randomBoolean()) { // random empty refresh
engine.refresh("test");
}
assertTrue("safe access should be required we carried it over", engine.isSafeAccessRequired());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
assertEquals(1, searcher.reader().numDocs());
TopDocs search = searcher.searcher().search(new MatchAllDocsQuery(), 1);
org.apache.lucene.document.Document luceneDoc = searcher.searcher().doc(search.scoreDocs[0].doc);
assertEquals("updated", luceneDoc.get("value"));
}
doc = testParsedDocument("2", null, testDocumentWithTextField("test"),
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
operation = randomBoolean() ?
appendOnlyPrimary(doc, false, 1)
: appendOnlyReplica(doc, false, 1, engine.seqNoService().generateSeqNo());
engine.index(operation);
assertTrue("safe access should be required", engine.isSafeAccessRequired());
assertEquals(1, engine.getVersionMapSize()); // now we add this to the map
engine.refresh("test");
if (randomBoolean()) { // randomly refresh here again
engine.refresh("test");
}
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
assertEquals(2, searcher.reader().numDocs());
}
assertFalse("safe access should NOT be required last indexing round was only append only", engine.isSafeAccessRequired());
engine.delete(new Engine.Delete(operation.type(), operation.id(), operation.uid()));
assertTrue("safe access should be required", engine.isSafeAccessRequired());
engine.refresh("test");
assertTrue("safe access should be required", engine.isSafeAccessRequired());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
assertEquals(1, searcher.reader().numDocs());
}
}
public void testSegments() throws Exception {
try (Store store = createStore();
InternalEngine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
@ -3364,6 +3428,7 @@ public class InternalEngineTests extends EngineTestCase {
}
Collections.shuffle(docs, random());
CountDownLatch startGun = new CountDownLatch(thread.length);
AtomicInteger offset = new AtomicInteger(-1);
for (int i = 0; i < thread.length; i++) {
thread[i] = new Thread() {
@ -3375,6 +3440,7 @@ public class InternalEngineTests extends EngineTestCase {
} catch (InterruptedException e) {
throw new AssertionError(e);
}
assertEquals(0, engine.getVersionMapSize());
int docOffset;
while ((docOffset = offset.incrementAndGet()) < docs.size()) {
try {
@ -3387,6 +3453,9 @@ public class InternalEngineTests extends EngineTestCase {
};
thread[i].start();
}
try (Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL)) {
assertEquals("unexpected refresh", 0, searcher.reader().maxDoc());
}
for (int i = 0; i < thread.length; i++) {
thread[i].join();
}
@ -4227,4 +4296,54 @@ public class InternalEngineTests extends EngineTestCase {
}
}
}
public void testConcurrentAppendUpdateAndRefresh() throws InterruptedException, IOException {
int numDocs = scaledRandomIntBetween(100, 1000);
CountDownLatch latch = new CountDownLatch(2);
AtomicBoolean done = new AtomicBoolean(false);
AtomicInteger numDeletes = new AtomicInteger();
Thread thread = new Thread(() -> {
try {
latch.countDown();
latch.await();
for (int j = 0; j < numDocs; j++) {
String docID = Integer.toString(j);
ParsedDocument doc = testParsedDocument(docID, null, testDocumentWithTextField(),
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
Engine.Index operation = appendOnlyPrimary(doc, false, 1);
engine.index(operation);
if (rarely()) {
engine.delete(new Engine.Delete(operation.type(), operation.id(), operation.uid()));
numDeletes.incrementAndGet();
} else {
doc = testParsedDocument(docID, null, testDocumentWithTextField("updated"),
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
Engine.Index update = indexForDoc(doc);
engine.index(update);
}
}
} catch (Exception e) {
throw new AssertionError(e);
} finally {
done.set(true);
}
});
thread.start();
latch.countDown();
latch.await();
while (done.get() == false) {
engine.refresh("test", Engine.SearcherScope.INTERNAL);
}
thread.join();
engine.refresh("test", Engine.SearcherScope.INTERNAL);
try (Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL)) {
TopDocs search = searcher.searcher().search(new MatchAllDocsQuery(), searcher.reader().numDocs());
for (int i = 0; i < search.scoreDocs.length; i++) {
org.apache.lucene.document.Document luceneDoc = searcher.searcher().doc(search.scoreDocs[i].doc);
assertEquals("updated", luceneDoc.get("value"));
}
int totalNumDocs = numDocs - numDeletes.get();
assertEquals(totalNumDocs, searcher.reader().numDocs());
}
}
}

View File

@ -217,4 +217,59 @@ public class LiveVersionMapTests extends ESTestCase {
assertTrue(versionValue instanceof DeleteVersionValue);
});
}
public void testCarryOnSafeAccess() throws IOException {
LiveVersionMap map = new LiveVersionMap();
assertFalse(map.isUnsafe());
assertFalse(map.isSafeAccessRequired());
map.enforceSafeAccess();
assertTrue(map.isSafeAccessRequired());
assertFalse(map.isUnsafe());
int numIters = randomIntBetween(1, 5);
for (int i = 0; i < numIters; i++) { // if we don't do anything ie. no adds etc we will stay with the safe access required
map.beforeRefresh();
map.afterRefresh(randomBoolean());
assertTrue("failed in iter: " + i, map.isSafeAccessRequired());
}
map.maybePutUnderLock(new BytesRef(""), new VersionValue(randomLong(), randomLong(), randomLong()));
assertFalse(map.isUnsafe());
assertEquals(1, map.getAllCurrent().size());
map.beforeRefresh();
map.afterRefresh(randomBoolean());
assertFalse(map.isUnsafe());
assertFalse(map.isSafeAccessRequired());
map.maybePutUnderLock(new BytesRef(""), new VersionValue(randomLong(), randomLong(), randomLong()));
assertTrue(map.isUnsafe());
assertFalse(map.isSafeAccessRequired());
assertEquals(0, map.getAllCurrent().size());
}
public void testRefreshTransition() throws IOException {
LiveVersionMap map = new LiveVersionMap();
map.maybePutUnderLock(uid("1"), new VersionValue(randomLong(), randomLong(), randomLong()));
assertTrue(map.isUnsafe());
assertNull(map.getUnderLock(uid("1")));
map.beforeRefresh();
assertTrue(map.isUnsafe());
assertNull(map.getUnderLock(uid("1")));
map.afterRefresh(randomBoolean());
assertNull(map.getUnderLock(uid("1")));
assertFalse(map.isUnsafe());
map.enforceSafeAccess();
map.maybePutUnderLock(uid("1"), new VersionValue(randomLong(), randomLong(), randomLong()));
assertFalse(map.isUnsafe());
assertNotNull(map.getUnderLock(uid("1")));
map.beforeRefresh();
assertFalse(map.isUnsafe());
assertTrue(map.isSafeAccessRequired());
assertNotNull(map.getUnderLock(uid("1")));
map.afterRefresh(randomBoolean());
assertNull(map.getUnderLock(uid("1")));
assertFalse(map.isUnsafe());
assertTrue(map.isSafeAccessRequired());
}
}