-
Notifications
You must be signed in to change notification settings - Fork 587
HDDS-12742. Make RDBStoreAbstractIterator Return Reference-Counted KeyValues #8203
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
Changes from 2 commits
30facb9
aab6ebc
5dd7460
e4bf8f7
9ab91c9
abe06ab
d215d4b
85c74b4
7d82a9e
ce6ab81
091e8ca
6bc5c86
1b394c2
7fdced2
f5b633b
1971a96
a33f265
0c3ae4f
fbe213b
71f9c28
5734027
99c508e
f16e1b9
86fe101
cfde81f
f825754
e4155a6
985d612
b98968b
78d33af
ed632a8
58f81cc
3f77cbe
1fecc85
deb0011
346a685
f6b651c
93c7f91
f81c116
7b37a29
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 |
|---|---|---|
|
|
@@ -21,6 +21,8 @@ | |
| import java.util.NoSuchElementException; | ||
| import java.util.function.Consumer; | ||
| import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; | ||
| import org.apache.ratis.util.ReferenceCountedObject; | ||
| import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -30,30 +32,32 @@ | |
| * @param <RAW> the raw type. | ||
| */ | ||
| abstract class RDBStoreAbstractIterator<RAW> | ||
| implements TableIterator<RAW, Table.KeyValue<RAW, RAW>> { | ||
| implements TableIterator<RAW, UncheckedAutoCloseableSupplier<RawKeyValue<RAW>>> { | ||
|
|
||
| private static final Logger LOG = | ||
| LoggerFactory.getLogger(RDBStoreAbstractIterator.class); | ||
|
|
||
| private final ManagedRocksIterator rocksDBIterator; | ||
| private final RDBTable rocksDBTable; | ||
| private Table.KeyValue<RAW, RAW> currentEntry; | ||
| private ReferenceCountedObject<RawKeyValue<RAW>> currentEntry; | ||
| // This is for schemas that use a fixed-length | ||
| // prefix for each key. | ||
| private final RAW prefix; | ||
| private boolean hasNext; | ||
| private boolean closed; | ||
|
|
||
| RDBStoreAbstractIterator(ManagedRocksIterator iterator, RDBTable table, | ||
| RAW prefix) { | ||
| this.rocksDBIterator = iterator; | ||
| this.rocksDBTable = table; | ||
| this.prefix = prefix; | ||
| this.currentEntry = null; | ||
| this.hasNext = false; | ||
| this.closed = false; | ||
| } | ||
|
|
||
| /** @return the key for the current entry. */ | ||
| abstract RAW key(); | ||
|
|
||
| /** @return the {@link Table.KeyValue} for the current entry. */ | ||
| abstract Table.KeyValue<RAW, RAW> getKeyValue(); | ||
| abstract ReferenceCountedObject<RawKeyValue<RAW>> getKeyValue(); | ||
|
|
||
| /** Seek to the given key. */ | ||
| abstract void seek0(RAW key); | ||
|
|
@@ -78,38 +82,50 @@ final RAW getPrefix() { | |
|
|
||
| @Override | ||
| public final void forEachRemaining( | ||
| Consumer<? super Table.KeyValue<RAW, RAW>> action) { | ||
| Consumer<? super UncheckedAutoCloseableSupplier<RawKeyValue<RAW>>> action) { | ||
| while (hasNext()) { | ||
| action.accept(next()); | ||
| UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> entry = next(); | ||
| action.accept(entry); | ||
| } | ||
| } | ||
|
|
||
| private void setCurrentEntry() { | ||
| if (rocksDBIterator.get().isValid()) { | ||
| if (currentEntry != null) { | ||
| currentEntry.release(); | ||
| } | ||
|
|
||
| boolean isValid = !closed && rocksDBIterator.get().isValid(); | ||
| if (isValid) { | ||
| currentEntry = getKeyValue(); | ||
| currentEntry.retain(); | ||
| } else { | ||
| currentEntry = null; | ||
| } | ||
| setHasNext(isValid, currentEntry); | ||
| } | ||
|
|
||
| public void setHasNext(boolean isValid, ReferenceCountedObject<RawKeyValue<RAW>> entry) { | ||
| this.hasNext = isValid && (prefix == null || startsWithPrefix(entry.get().getKey())); | ||
| } | ||
|
|
||
| @Override | ||
| public final boolean hasNext() { | ||
| return rocksDBIterator.get().isValid() && | ||
| (prefix == null || startsWithPrefix(key())); | ||
| return hasNext; | ||
| } | ||
|
|
||
| @Override | ||
| public final Table.KeyValue<RAW, RAW> next() { | ||
| setCurrentEntry(); | ||
| if (currentEntry != null) { | ||
| public final synchronized UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> next() { | ||
|
||
| if (hasNext()) { | ||
| UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> entry = currentEntry.retainAndReleaseOnClose(); | ||
| rocksDBIterator.get().next(); | ||
|
Contributor
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. rocksDbIterator.get().next() can also be moved to setCurrentEntry(), as we are getting next entry, updating hasNext flag together.
Contributor
Author
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. no we cannot do that. What about for the first entry. You shouldn't call next for the first entry in the iterator |
||
| return currentEntry; | ||
| setCurrentEntry(); | ||
| return entry; | ||
| } | ||
| throw new NoSuchElementException("RocksDB Store has no more elements"); | ||
| } | ||
|
|
||
| @Override | ||
| public final void seekToFirst() { | ||
| public final synchronized void seekToFirst() { | ||
| if (prefix == null) { | ||
| rocksDBIterator.get().seekToFirst(); | ||
| } else { | ||
|
|
@@ -119,7 +135,7 @@ public final void seekToFirst() { | |
| } | ||
|
|
||
| @Override | ||
| public final void seekToLast() { | ||
| public final synchronized void seekToLast() { | ||
| if (prefix == null) { | ||
| rocksDBIterator.get().seekToLast(); | ||
| } else { | ||
|
|
@@ -129,26 +145,29 @@ public final void seekToLast() { | |
| } | ||
|
|
||
| @Override | ||
| public final Table.KeyValue<RAW, RAW> seek(RAW key) { | ||
| public final synchronized UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> seek(RAW key) { | ||
| seek0(key); | ||
| setCurrentEntry(); | ||
| return currentEntry; | ||
| // Current entry should be only closed when the next() and thus closing the returned entry should be a noop. | ||
| return currentEntry.retainAndReleaseOnClose(); | ||
| } | ||
|
|
||
| @Override | ||
| public final void removeFromDB() throws IOException { | ||
| public final synchronized void removeFromDB() throws IOException { | ||
| if (rocksDBTable == null) { | ||
| throw new UnsupportedOperationException("remove"); | ||
| } | ||
| if (currentEntry != null) { | ||
| delete(currentEntry.getKey()); | ||
| if (currentEntry.get() != null) { | ||
| delete(currentEntry.get().getKey()); | ||
| } else { | ||
| LOG.info("Failed to removeFromDB: currentEntry == null"); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| public synchronized void close() { | ||
| rocksDBIterator.close(); | ||
| closed = true; | ||
| setCurrentEntry(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,13 +18,21 @@ | |
| package org.apache.hadoop.hdds.utils.db; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.HashSet; | ||
| import java.util.Objects; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.Stack; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.function.Function; | ||
| import org.apache.commons.lang3.exception.UncheckedInterruptedException; | ||
| import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; | ||
| import org.apache.ratis.util.Preconditions; | ||
| import org.apache.ratis.util.ReferenceCountedObject; | ||
|
|
||
| /** | ||
| * Implement {@link RDBStoreAbstractIterator} using {@link CodecBuffer}. | ||
| * Any Key or Value returned will be only valid within the lifecycle of this iterator. | ||
| */ | ||
swamirishi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| class RDBStoreCodecBufferIterator | ||
| extends RDBStoreAbstractIterator<CodecBuffer> { | ||
|
|
@@ -84,38 +92,77 @@ CodecBuffer getFromDb() { | |
| } | ||
| } | ||
|
|
||
| private final Buffer keyBuffer; | ||
| private final Buffer valueBuffer; | ||
| private final Stack<RawKeyValue<Buffer>> availableBufferStack; | ||
|
||
| private final Set<RawKeyValue<Buffer>> inUseBuffers; | ||
| private final Object bufferLock; | ||
szetszwo marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| private final AtomicBoolean closed = new AtomicBoolean(); | ||
|
|
||
| RDBStoreCodecBufferIterator(ManagedRocksIterator iterator, RDBTable table, | ||
| CodecBuffer prefix) { | ||
| CodecBuffer prefix, int maxNumberOfBuffersInMemory) { | ||
| super(iterator, table, prefix); | ||
|
|
||
| // We need atleast 2 buffers one for setting next value and one for sending the current value. | ||
| maxNumberOfBuffersInMemory = Math.max(2, maxNumberOfBuffersInMemory); | ||
| final String name = table != null ? table.getName() : null; | ||
| this.keyBuffer = new Buffer( | ||
| new CodecBuffer.Capacity(name + "-iterator-key", 1 << 10), | ||
| buffer -> getRocksDBIterator().get().key(buffer)); | ||
| this.valueBuffer = new Buffer( | ||
| new CodecBuffer.Capacity(name + "-iterator-value", 4 << 10), | ||
| buffer -> getRocksDBIterator().get().value(buffer)); | ||
| this.availableBufferStack = new Stack<>(); | ||
| this.inUseBuffers = new HashSet<>(); | ||
| for (int i = 0; i < maxNumberOfBuffersInMemory; i++) { | ||
| Buffer keyBuffer = new Buffer( | ||
| new CodecBuffer.Capacity(name + "-iterator-key-" + i, 1 << 10), | ||
| buffer -> getRocksDBIterator().get().key(buffer)); | ||
| Buffer valueBuffer = new Buffer( | ||
| new CodecBuffer.Capacity(name + "-iterator-value-" + i, 4 << 10), | ||
| buffer -> getRocksDBIterator().get().value(buffer)); | ||
| availableBufferStack.push(new RawKeyValue<>(keyBuffer, valueBuffer)); | ||
| } | ||
|
|
||
| this.bufferLock = new Object(); | ||
| seekToFirst(); | ||
| } | ||
|
|
||
| void assertOpen() { | ||
| Preconditions.assertTrue(!closed.get(), "Already closed"); | ||
| } | ||
|
|
||
| @Override | ||
| CodecBuffer key() { | ||
| assertOpen(); | ||
| return keyBuffer.getFromDb(); | ||
| private <V> V getFromStack(Object lock, Stack<V> stack, Set<V> inUseSet) { | ||
| synchronized (Objects.requireNonNull(lock)) { | ||
| while (stack.isEmpty()) { | ||
| try { | ||
| assertOpen(); | ||
| lock.wait(1000); | ||
| } catch (InterruptedException e) { | ||
| throw new UncheckedInterruptedException(e); | ||
| } | ||
| } | ||
| V popped = stack.pop(); | ||
| inUseSet.add(popped); | ||
| return popped; | ||
| } | ||
| } | ||
|
|
||
| private ReferenceCountedObject<RawKeyValue<CodecBuffer>> getReferenceCountedBuffer( | ||
| RawKeyValue<Buffer> key, Stack<RawKeyValue<Buffer>> stack, Set<RawKeyValue<Buffer>> inUseSet, | ||
| Object lock, Function<RawKeyValue<Buffer>, RawKeyValue<CodecBuffer>> transformer) { | ||
| RawKeyValue<CodecBuffer> value = transformer.apply(key); | ||
| return ReferenceCountedObject.wrap(value, () -> { | ||
| }, completelyReleased -> { | ||
| if (!completelyReleased) { | ||
| return; | ||
| } | ||
| synchronized (lock) { | ||
| stack.push(key); | ||
| inUseSet.remove(key); | ||
| lock.notify(); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| @Override | ||
| Table.KeyValue<CodecBuffer, CodecBuffer> getKeyValue() { | ||
| ReferenceCountedObject<RawKeyValue<CodecBuffer>> getKeyValue() { | ||
| assertOpen(); | ||
| return Table.newKeyValue(key(), valueBuffer.getFromDb()); | ||
| RawKeyValue<Buffer> kvBuffer = getFromStack(bufferLock, availableBufferStack, inUseBuffers); | ||
| Function<RawKeyValue<Buffer>, RawKeyValue<CodecBuffer>> transformer = | ||
| kv -> new RawKeyValue<>(kv.getKey().getFromDb(), kv.getValue().getFromDb()); | ||
| return getReferenceCountedBuffer(kvBuffer, availableBufferStack, inUseBuffers, bufferLock, transformer); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -143,13 +190,29 @@ boolean startsWithPrefix(CodecBuffer key) { | |
| return key.startsWith(prefix); | ||
| } | ||
|
|
||
| private <V> void release(Stack<V> valueStack, Set<V> inUseSet, Object lock, Function<V, Void> releaser) { | ||
| synchronized (Objects.requireNonNull(lock)) { | ||
| while (!valueStack.isEmpty()) { | ||
| V popped = valueStack.pop(); | ||
| releaser.apply(popped); | ||
| } | ||
|
|
||
| for (V inUseValue : inUseSet) { | ||
| releaser.apply(inUseValue); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| public synchronized void close() { | ||
| if (closed.compareAndSet(false, true)) { | ||
| super.close(); | ||
| Optional.ofNullable(getPrefix()).ifPresent(CodecBuffer::release); | ||
| keyBuffer.release(); | ||
| valueBuffer.release(); | ||
| release(availableBufferStack, inUseBuffers, bufferLock, kv -> { | ||
| kv.getKey().release(); | ||
| kv.getValue().release(); | ||
| return null; | ||
| }); | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.