Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -117,8 +117,6 @@ public class InternalEngine extends Engine {
// we use the hashed variant since we iterate over it and check removal and additions on existing keys
private final LiveVersionMap versionMap = new LiveVersionMap();

private final KeyedLock<BytesRef> keyedLock = new KeyedLock<>();

private volatile SegmentInfos lastCommittedSegmentInfos;

private final IndexThrottle throttle;
Expand Down Expand Up @@ -551,7 +549,11 @@ public GetResult get(Get get, BiFunction<String, SearcherScope, Searcher> search
ensureOpen();
SearcherScope scope;
if (get.realtime()) {
VersionValue versionValue = getVersionFromMap(get.uid().bytes());
VersionValue versionValue = null;
try (Releasable ignore = versionMap.acquireLock(get.uid().bytes())) {
// we need to lock here to access the version map to do this truly in RT
versionValue = getVersionFromMap(get.uid().bytes());
}
if (versionValue != null) {
if (versionValue.isDelete()) {
return GetResult.NOT_EXISTS;
Expand Down Expand Up @@ -733,7 +735,7 @@ public IndexResult index(Index index) throws IOException {
ensureOpen();
assert assertIncomingSequenceNumber(index.origin(), index.seqNo());
assert assertVersionType(index);
try (Releasable ignored = acquireLock(index.uid());
try (Releasable ignored = versionMap.acquireLock(index.uid().bytes());
Releasable indexThrottle = doThrottle ? () -> {} : throttle.acquireThrottle()) {
lastWriteNanos = index.startTime();
/* A NOTE ABOUT APPEND ONLY OPTIMIZATIONS:
Expand Down Expand Up @@ -1059,7 +1061,7 @@ public DeleteResult delete(Delete delete) throws IOException {
assert assertIncomingSequenceNumber(delete.origin(), delete.seqNo());
final DeleteResult deleteResult;
// NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments:
try (ReleasableLock ignored = readLock.acquire(); Releasable ignored2 = acquireLock(delete.uid())) {
try (ReleasableLock ignored = readLock.acquire(); Releasable ignored2 = versionMap.acquireLock(delete.uid().bytes())) {
ensureOpen();
lastWriteNanos = delete.startTime();
final DeletionStrategy plan;
Expand Down Expand Up @@ -1520,7 +1522,8 @@ private void pruneDeletedTombstones() {
// we only need to prune the deletes map; the current/old version maps are cleared on refresh:
for (Map.Entry<BytesRef, DeleteVersionValue> entry : versionMap.getAllTombstones()) {
BytesRef uid = entry.getKey();
try (Releasable ignored = acquireLock(uid)) { // can we do it without this lock on each value? maybe batch to a set and get the lock once per set?
try (Releasable ignored = versionMap.acquireLock(uid)) {
// can we do it without this lock on each value? maybe batch to a set and get the lock once per set?

// Must re-get it here, vs using entry.getValue(), in case the uid was indexed/deleted since we pulled the iterator:
DeleteVersionValue versionValue = versionMap.getTombstoneUnderLock(uid);
Expand Down Expand Up @@ -1767,14 +1770,6 @@ protected ReferenceManager<IndexSearcher> getSearcherManager(String source, Sear
}
}

private Releasable acquireLock(BytesRef uid) {
return keyedLock.acquire(uid);
}

private Releasable acquireLock(Term uid) {
return acquireLock(uid.bytes());
}

private long loadCurrentVersionFromIndex(Term uid) throws IOException {
assert incrementIndexVersionLookup();
try (Searcher searcher = acquireSearcher("load_version", SearcherScope.INTERNAL)) {
Expand Down
104 changes: 77 additions & 27 deletions core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,9 @@
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.KeyedLock;

import java.io.IOException;
import java.util.Collection;
Expand All @@ -34,6 +36,8 @@
/** Maps _uid value to its version information. */
final class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {

private final KeyedLock<BytesRef> keyedLock = new KeyedLock<>();

/**
* Resets the internal map and adjusts it's capacity as if there were no indexing operations.
* This must be called under write lock in the engine
Expand All @@ -49,7 +53,7 @@ void adjustMapSizeUnderLock() {
private static final class VersionLookup {

private static final VersionLookup EMPTY = new VersionLookup(Collections.emptyMap());
private final Map<BytesRef,VersionValue> map;
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
Expand Down Expand Up @@ -144,31 +148,36 @@ Maps invalidateOldMap() {
}

// All deletes also go here, and delete "tombstones" are retained after refresh:
private final Map<BytesRef,DeleteVersionValue> tombstones = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
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:
/**
* Bytes consumed for each BytesRef UID:
* In this base value, we account for the {@link BytesRef} object itself as
* well as the header of the byte[] array it holds, and some lost bytes due
* to object alignment. So consumers of this constant just have to add the
* length of the byte[] (assuming it is not shared between multiple
* instances). */
* instances).
*/
private static final long BASE_BYTES_PER_BYTESREF =
// shallow memory usage of the BytesRef object
RamUsageEstimator.shallowSizeOfInstance(BytesRef.class) +
// shallow memory usage of the BytesRef object
RamUsageEstimator.shallowSizeOfInstance(BytesRef.class) +
// header of the byte[] array
RamUsageEstimator.NUM_BYTES_ARRAY_HEADER +
// with an alignment size (-XX:ObjectAlignmentInBytes) of 8 (default),
// there could be between 0 and 7 lost bytes, so we account for 3
// lost bytes on average
3;

/** Bytes used by having CHM point to a key/value. */
/**
* Bytes used by having CHM point to a key/value.
*/
private static final long BASE_BYTES_PER_CHM_ENTRY;

static {
// use the same impl as the Maps does
Map<Integer, Integer> map = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
Expand All @@ -181,11 +190,15 @@ Maps invalidateOldMap() {
BASE_BYTES_PER_CHM_ENTRY = chmEntryShallowSize + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
}

/** Tracks bytes used by current map, i.e. what is freed on refresh. For deletes, which are also added to tombstones, we only account
* for the CHM entry here, and account for BytesRef/VersionValue against the tombstones, since refresh would not clear this RAM. */
/**
* Tracks bytes used by current map, i.e. what is freed on refresh. For deletes, which are also added to tombstones, we only account
* for the CHM entry here, and account for BytesRef/VersionValue against the tombstones, since refresh would not clear this RAM.
*/
final AtomicLong ramBytesUsedCurrent = new AtomicLong();

/** Tracks bytes used by tombstones (deletes) */
/**
* Tracks bytes used by tombstones (deletes)
*/
final AtomicLong ramBytesUsedTombstones = new AtomicLong();

@Override
Expand Down Expand Up @@ -215,12 +228,15 @@ public void afterRefresh(boolean didRefresh) throws IOException {

}

/** Returns the live version (add or delete) for this uid. */
/**
* Returns the live version (add or delete) for this uid.
*/
VersionValue getUnderLock(final BytesRef uid) {
return getUnderLock(uid, maps);
}

private VersionValue getUnderLock(final BytesRef uid, Maps currentMaps) {
assert keyedLock.isHeldByCurrentThread(uid);
// First try to get the "live" value:
VersionValue value = currentMaps.current.get(uid);
if (value != null) {
Expand Down Expand Up @@ -255,8 +271,11 @@ boolean isSafeAccessRequired() {
return maps.isSafeAccessMode();
}

/** Adds this uid/version to the pending adds map iff the map needs safe access. */
/**
* Adds this uid/version to the pending adds map iff the map needs safe access.
*/
void maybePutUnderLock(BytesRef uid, VersionValue version) {
assert keyedLock.isHeldByCurrentThread(uid);
Maps maps = this.maps;
if (maps.isSafeAccessMode()) {
putUnderLock(uid, version, maps);
Expand All @@ -271,14 +290,19 @@ private boolean putAssertionMap(BytesRef uid, VersionValue version) {
return true;
}

/** Adds this uid/version to the pending adds map. */
/**
* 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. */
/**
* Adds this uid/version to the pending adds map.
*/
private void putUnderLock(BytesRef uid, VersionValue version, Maps maps) {
assert keyedLock.isHeldByCurrentThread(uid);
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);
Expand All @@ -301,7 +325,7 @@ private void putUnderLock(BytesRef uid, VersionValue version, Maps maps) {
final VersionValue prevTombstone;
if (version.isDelete()) {
// Also enroll the delete into tombstones, and account for its RAM too:
prevTombstone = tombstones.put(uid, (DeleteVersionValue)version);
prevTombstone = tombstones.put(uid, (DeleteVersionValue) version);

// We initially account for BytesRef/VersionValue RAM for a delete against the tombstones, because this RAM will not be freed up
// on refresh. Later, in removeTombstoneUnderLock, if we clear the tombstone entry but the delete remains in current, we shift
Expand All @@ -321,20 +345,22 @@ private void putUnderLock(BytesRef uid, VersionValue version, Maps maps) {
// Deduct tombstones bytes used for the version we just removed or replaced:
if (prevTombstone != null) {
long v = ramBytesUsedTombstones.addAndGet(-(BASE_BYTES_PER_CHM_ENTRY + prevTombstone.ramBytesUsed() + uidRAMBytesUsed));
assert v >= 0: "bytes=" + v;
assert v >= 0 : "bytes=" + v;
}
}

/** Removes this uid from the pending deletes map. */
/**
* Removes this uid from the pending deletes map.
*/
void removeTombstoneUnderLock(BytesRef uid) {

assert keyedLock.isHeldByCurrentThread(uid);
long uidRAMBytesUsed = BASE_BYTES_PER_BYTESREF + uid.bytes.length;

final VersionValue prev = tombstones.remove(uid);
if (prev != null) {
assert prev.isDelete();
long v = ramBytesUsedTombstones.addAndGet(-(BASE_BYTES_PER_CHM_ENTRY + prev.ramBytesUsed() + uidRAMBytesUsed));
assert v >= 0: "bytes=" + v;
assert v >= 0 : "bytes=" + v;
}
final VersionValue curVersion = maps.current.get(uid);
if (curVersion != null && curVersion.isDelete()) {
Expand All @@ -345,22 +371,31 @@ void removeTombstoneUnderLock(BytesRef uid) {
}
}

/** Caller has a lock, so that this uid will not be concurrently added/deleted by another thread. */
/**
* Caller has a lock, so that this uid will not be concurrently added/deleted by another thread.
*/
DeleteVersionValue getTombstoneUnderLock(BytesRef uid) {
assert keyedLock.isHeldByCurrentThread(uid);
return tombstones.get(uid);
}

/** Iterates over all deleted versions, including new ones (not yet exposed via reader) and old ones (exposed via reader but not yet GC'd). */
/**
* Iterates over all deleted versions, including new ones (not yet exposed via reader) and old ones (exposed via reader but not yet GC'd).
*/
Iterable<Map.Entry<BytesRef, DeleteVersionValue>> getAllTombstones() {
return tombstones.entrySet();
}

/** clears all tombstones ops */
/**
* clears all tombstones ops
*/
void clearTombstones() {
tombstones.clear();
}

/** Called when this index is closed. */
/**
* Called when this index is closed.
*/
synchronized void clear() {
maps = new Maps();
tombstones.clear();
Expand All @@ -377,8 +412,10 @@ public long ramBytesUsed() {
return ramBytesUsedCurrent.get() + ramBytesUsedTombstones.get();
}

/** Returns how much RAM would be freed up by refreshing. This is {@link #ramBytesUsed} except does not include tombstones because they
* don't clear on refresh. */
/**
* Returns how much RAM would be freed up by refreshing. This is {@link #ramBytesUsed} except does not include tombstones because they
* don't clear on refresh.
*/
long ramBytesUsedForRefresh() {
return ramBytesUsedCurrent.get();
}
Expand All @@ -389,7 +426,20 @@ public Collection<Accountable> getChildResources() {
return Collections.emptyList();
}

/** Returns the current internal versions as a point in time snapshot*/
/**
* Returns the current internal versions as a point in time snapshot
*/
Map<BytesRef, VersionValue> getAllCurrent() {
return maps.current.map;
}}
}

/**
* Acquires a releaseable lock for the given uId. All *UnderLock methods require
* this lock to be hold by the caller otherwise the visibility guarantees of this version
* map are broken. We assert on this lock to be hold when calling these methods.
* @see KeyedLock
*/
Releasable acquireLock(BytesRef uid) {
return keyedLock.acquire(uid);
}
}
Loading