From 30facb906516f6e7d0e9d7ee43d1cd030687b321 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sat, 12 Apr 2025 10:32:15 -0700 Subject: [PATCH 01/31] HDDS-12742. Make RDBStoreAbstractIterator return a reference counted KeyValue Change-Id: Ifd24aadf501f3af1f1385ea2a43b036ddee2c5a1 --- .../utils/db/RDBStoreAbstractIterator.java | 65 ++++--- .../utils/db/RDBStoreByteArrayIterator.java | 11 +- .../utils/db/RDBStoreCodecBufferIterator.java | 104 +++++++++--- .../apache/hadoop/hdds/utils/db/RDBTable.java | 54 +++++- .../hadoop/hdds/utils/db/RawKeyValue.java | 4 +- .../hadoop/hdds/utils/db/TypedTable.java | 29 ++-- .../db/TestRDBStoreByteArrayIterator.java | 128 +++++++------- .../db/TestRDBStoreCodecBufferIterator.java | 158 ++++++++++++++---- 8 files changed, 388 insertions(+), 165 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index 75104a55ed72..686e083904c7 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -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 the raw type. */ abstract class RDBStoreAbstractIterator - implements TableIterator> { + implements TableIterator>> { private static final Logger LOG = LoggerFactory.getLogger(RDBStoreAbstractIterator.class); private final ManagedRocksIterator rocksDBIterator; private final RDBTable rocksDBTable; - private Table.KeyValue currentEntry; + private ReferenceCountedObject> 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 getKeyValue(); + abstract ReferenceCountedObject> getKeyValue(); /** Seek to the given key. */ abstract void seek0(RAW key); @@ -78,38 +82,50 @@ final RAW getPrefix() { @Override public final void forEachRemaining( - Consumer> action) { + Consumer>> action) { while (hasNext()) { - action.accept(next()); + UncheckedAutoCloseableSupplier> 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> 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 next() { - setCurrentEntry(); - if (currentEntry != null) { + public final synchronized UncheckedAutoCloseableSupplier> next() { + if (hasNext()) { + UncheckedAutoCloseableSupplier> entry = currentEntry.retainAndReleaseOnClose(); rocksDBIterator.get().next(); - 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 seek(RAW key) { + public final synchronized UncheckedAutoCloseableSupplier> 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(); } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java index 62303c8a3bb7..6b653f7eadab 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; +import org.apache.ratis.util.ReferenceCountedObject; /** * RocksDB store iterator using the byte[] API. @@ -33,14 +34,10 @@ class RDBStoreByteArrayIterator extends RDBStoreAbstractIterator { } @Override - byte[] key() { - return getRocksDBIterator().get().key(); - } - - @Override - Table.KeyValue getKeyValue() { + ReferenceCountedObject> getKeyValue() { final ManagedRocksIterator i = getRocksDBIterator(); - return RawKeyValue.create(i.get().key(), i.get().value()); + RawKeyValue rawKV = RawKeyValue.create(i.get().key(), i.get().value()); + return ReferenceCountedObject.wrap(rawKV); } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 59c6661129ef..cb1fadcba6db 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -18,16 +18,27 @@ 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implement {@link RDBStoreAbstractIterator} using {@link CodecBuffer}. + * Any Key or Value returned will be only valid within the lifecycle of this iterator. */ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator { + private static final Logger LOG = LoggerFactory.getLogger(RDBStoreCodecBufferIterator.class); static class Buffer { private final CodecBuffer.Capacity initialCapacity; private final PutToByteBuffer source; @@ -84,21 +95,30 @@ CodecBuffer getFromDb() { } } - private final Buffer keyBuffer; - private final Buffer valueBuffer; + private final Stack> availableBufferStack; + private final Set> inUseBuffers; + private final Object bufferLock; 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(); } @@ -106,16 +126,46 @@ void assertOpen() { Preconditions.assertTrue(!closed.get(), "Already closed"); } - @Override - CodecBuffer key() { - assertOpen(); - return keyBuffer.getFromDb(); + private V getFromStack(Object lock, Stack stack, Set 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> getReferenceCountedBuffer( + RawKeyValue key, Stack> stack, Set> inUseSet, + Object lock, Function, RawKeyValue> transformer) { + RawKeyValue 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 getKeyValue() { + ReferenceCountedObject> getKeyValue() { assertOpen(); - return Table.newKeyValue(key(), valueBuffer.getFromDb()); + RawKeyValue kvBuffer = getFromStack(bufferLock, availableBufferStack, inUseBuffers); + Function, RawKeyValue> transformer = + kv -> new RawKeyValue<>(kv.getKey().getFromDb(), kv.getValue().getFromDb()); + return getReferenceCountedBuffer(kvBuffer, availableBufferStack, inUseBuffers, bufferLock, transformer); } @Override @@ -143,13 +193,29 @@ boolean startsWithPrefix(CodecBuffer key) { return key.startsWith(prefix); } + private void release(Stack valueStack, Set inUseSet, Object lock, Function 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; + }); } } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index 9ce8e03f69a4..4ee9755a4223 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily; import org.apache.hadoop.util.Time; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -221,14 +222,59 @@ public TableIterator> iterator() @Override public TableIterator> iterator(byte[] prefix) throws IOException { - return new RDBStoreByteArrayIterator(db.newIterator(family, false), this, - prefix); + TableIterator>> itr = + new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); + return new TableIterator>() { + @Override + public void seekToFirst() { + itr.seekToFirst(); + } + + @Override + public void seekToLast() { + itr.seekToLast(); + } + + @Override + public KeyValue seek(byte[] bytes) throws IOException { + try (UncheckedAutoCloseableSupplier> kv = itr.seek(bytes)) { + return kv.get(); + } + } + + @Override + public void removeFromDB() { + itr.remove(); + } + + @Override + public void close() throws IOException { + itr.close(); + } + + @Override + public boolean hasNext() { + return itr.hasNext(); + } + + @Override + public KeyValue next() { + try (UncheckedAutoCloseableSupplier> kv = itr.next()) { + return kv.get(); + } + } + }; + } + + public TableIterator>> closeableSupplierIterator( + byte[] prefix) throws IOException { + return new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); } - TableIterator> iterator( + TableIterator>> iterator( CodecBuffer prefix) throws IOException { return new RDBStoreCodecBufferIterator(db.newIterator(family, false), - this, prefix); + this, prefix, 2); } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawKeyValue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawKeyValue.java index 7911314b114c..743bf44924f3 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawKeyValue.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawKeyValue.java @@ -25,7 +25,7 @@ * * @param The raw type. */ -public abstract class RawKeyValue implements KeyValue { +public class RawKeyValue implements KeyValue { /** * Create a KeyValue pair. * @@ -61,7 +61,7 @@ public byte[] getValue() { private final RAW key; private final RAW value; - private RawKeyValue(RAW key, RAW value) { + public RawKeyValue(RAW key, RAW value) { this.key = key; this.value = value; } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index f39d55327aab..cbd4385e98bd 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdds.utils.db.cache.TableNoCache; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.function.CheckedBiFunction; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; /** * Strongly typed table implementation. @@ -440,7 +441,7 @@ public Table.KeyValueIterator iterator(KEY prefix) } } else { final byte[] prefixBytes = encodeKey(prefix); - return new TypedTableIterator(rawTable.iterator(prefixBytes)); + return new TypedTableIterator(rawTable.closeableSupplierIterator(prefixBytes)); } } @@ -581,7 +582,7 @@ public VALUE getValue() throws IOException { } RawIterator newCodecBufferTableIterator( - TableIterator> i) { + TableIterator>> i) { return new RawIterator(i) { @Override AutoCloseSupplier convert(KEY key) throws IOException { @@ -600,7 +601,7 @@ public CodecBuffer get() { } @Override - KeyValue convert(KeyValue raw) + KeyValue convert(RawKeyValue raw) throws IOException { final int rawSize = raw.getValue().readableBytes(); final KEY key = keyCodec.fromCodecBuffer(raw.getKey()); @@ -615,7 +616,7 @@ KeyValue convert(KeyValue raw) */ public class TypedTableIterator extends RawIterator { TypedTableIterator( - TableIterator> rawIterator) { + TableIterator>> rawIterator) { super(rawIterator); } @@ -626,7 +627,7 @@ AutoCloseSupplier convert(KEY key) throws IOException { } @Override - KeyValue convert(KeyValue raw) { + KeyValue convert(RawKeyValue raw) { return new TypedKeyValue(raw); } } @@ -638,9 +639,9 @@ KeyValue convert(KeyValue raw) { */ abstract class RawIterator implements Table.KeyValueIterator { - private final TableIterator> rawIterator; + private final TableIterator>> rawIterator; - RawIterator(TableIterator> rawIterator) { + RawIterator(TableIterator>> rawIterator) { this.rawIterator = rawIterator; } @@ -648,10 +649,10 @@ abstract class RawIterator abstract AutoCloseSupplier convert(KEY key) throws IOException; /** - * Covert the given {@link Table.KeyValue} + * Covert the given {@link RawKeyValue} * from ({@link RAW}, {@link RAW}) to ({@link KEY}, {@link VALUE}). */ - abstract KeyValue convert(KeyValue raw) + abstract KeyValue convert(RawKeyValue raw) throws IOException; @Override @@ -666,9 +667,9 @@ public void seekToLast() { @Override public KeyValue seek(KEY key) throws IOException { - try (AutoCloseSupplier rawKey = convert(key)) { - final KeyValue result = rawIterator.seek(rawKey.get()); - return result == null ? null : convert(result); + try (AutoCloseSupplier rawKey = convert(key); + UncheckedAutoCloseableSupplier> result = rawIterator.seek(rawKey.get())) { + return result == null ? null : convert(result.get()); } } @@ -684,8 +685,8 @@ public boolean hasNext() { @Override public KeyValue next() { - try { - return convert(rawIterator.next()); + try (UncheckedAutoCloseableSupplier> kv = rawIterator.next()) { + return convert(kv.get()); } catch (IOException e) { throw new IllegalStateException("Failed next()", e); } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index 7a11f55720cb..6b6674f87fee 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksObjectUtils; import org.apache.log4j.Level; import org.apache.log4j.Logger; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -77,49 +78,52 @@ RDBStoreByteArrayIterator newIterator(byte[] prefix) { } @Test - public void testForeachRemainingCallsConsumerWithAllElements() { + public void testForeachRemainingCallsConsumerWithAllElements() throws IOException { when(rocksDBIteratorMock.isValid()) - .thenReturn(true, true, true, true, true, true, true, false); + .thenReturn(true, true, true, false); when(rocksDBIteratorMock.key()) - .thenReturn(new byte[]{0x00}, new byte[]{0x00}, new byte[]{0x01}, + .thenReturn(new byte[]{0x00}, new byte[]{0x01}, new byte[]{0x02}) .thenThrow(new NoSuchElementException()); when(rocksDBIteratorMock.value()) - .thenReturn(new byte[]{0x7f}, new byte[]{0x7f}, new byte[]{0x7e}, - new byte[]{0x7d}) + .thenReturn(new byte[]{0x7f}, new byte[]{0x7e}, new byte[]{0x7d}) .thenThrow(new NoSuchElementException()); - final Consumer> consumerStub + final Consumer>> consumerStub = mock(Consumer.class); - RDBStoreByteArrayIterator iter = newIterator(); - iter.forEachRemaining(consumerStub); - - ArgumentCaptor capture = - forClass(RawKeyValue.ByteArray.class); - verify(consumerStub, times(3)).accept(capture.capture()); - assertArrayEquals( - new byte[]{0x00}, capture.getAllValues().get(0).getKey()); - assertArrayEquals( - new byte[]{0x7f}, capture.getAllValues().get(0).getValue()); - assertArrayEquals( - new byte[]{0x01}, capture.getAllValues().get(1).getKey()); - assertArrayEquals( - new byte[]{0x7e}, capture.getAllValues().get(1).getValue()); - assertArrayEquals( - new byte[]{0x02}, capture.getAllValues().get(2).getKey()); - assertArrayEquals( - new byte[]{0x7d}, capture.getAllValues().get(2).getValue()); + try (RDBStoreByteArrayIterator iter = newIterator()) { + iter.forEachRemaining(consumerStub); + ArgumentCaptor>> capture = + forClass(UncheckedAutoCloseableSupplier.class); + verify(consumerStub, times(3)).accept(capture.capture()); + assertArrayEquals( + new byte[]{0x00}, capture.getAllValues().get(0).get().getKey()); + assertArrayEquals( + new byte[]{0x7f}, capture.getAllValues().get(0).get().getValue()); + assertArrayEquals( + new byte[]{0x01}, capture.getAllValues().get(1).get().getKey()); + assertArrayEquals( + new byte[]{0x7e}, capture.getAllValues().get(1).get().getValue()); + assertArrayEquals( + new byte[]{0x02}, capture.getAllValues().get(2).get().getKey()); + assertArrayEquals( + new byte[]{0x7d}, capture.getAllValues().get(2).get().getValue()); + } } @Test - public void testHasNextDependsOnIsvalid() { - when(rocksDBIteratorMock.isValid()).thenReturn(true, true, false); - - RDBStoreByteArrayIterator iter = newIterator(); - - assertTrue(iter.hasNext()); - assertFalse(iter.hasNext()); + public void testHasNextDoesNotDependsOnIsvalid() { + when(rocksDBIteratorMock.isValid()).thenReturn(true, false); + + try (RDBStoreByteArrayIterator iter = newIterator()) { + assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); + iter.next(); + assertFalse(iter.hasNext()); + assertThrows(NoSuchElementException.class, iter::next); + assertFalse(iter.hasNext()); + } } @Test @@ -169,7 +173,7 @@ public void testSeekReturnsTheActualKey() throws Exception { when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x7f}); RDBStoreByteArrayIterator iter = newIterator(); - final Table.KeyValue val = iter.seek(new byte[]{0x55}); + final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[]{0x55}); InOrder verifier = inOrder(rocksDBIteratorMock); @@ -179,8 +183,8 @@ public void testSeekReturnsTheActualKey() throws Exception { verifier.verify(rocksDBIteratorMock, times(1)).isValid(); verifier.verify(rocksDBIteratorMock, times(1)).key(); verifier.verify(rocksDBIteratorMock, times(1)).value(); - assertArrayEquals(new byte[]{0x00}, val.getKey()); - assertArrayEquals(new byte[]{0x7f}, val.getValue()); + assertArrayEquals(new byte[]{0x00}, val.get().getKey()); + assertArrayEquals(new byte[]{0x7f}, val.get().getValue()); } @Test @@ -191,8 +195,9 @@ public void testGettingTheKeyIfIteratorIsValid() throws Exception { RDBStoreByteArrayIterator iter = newIterator(); byte[] key = null; if (iter.hasNext()) { - final Table.KeyValue entry = iter.next(); - key = entry.getKey(); + try (UncheckedAutoCloseableSupplier> entry = iter.next()) { + key = entry.get().getKey(); + } } InOrder verifier = inOrder(rocksDBIteratorMock); @@ -208,22 +213,22 @@ public void testGettingTheValueIfIteratorIsValid() throws Exception { when(rocksDBIteratorMock.key()).thenReturn(new byte[]{0x00}); when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x7f}); - RDBStoreByteArrayIterator iter = newIterator(); - Table.KeyValue entry; - byte[] key = null; - byte[] value = null; - if (iter.hasNext()) { - entry = iter.next(); - key = entry.getKey(); - value = entry.getValue(); + try (RDBStoreByteArrayIterator iter = newIterator()) { + UncheckedAutoCloseableSupplier> entry; + byte[] key = null; + byte[] value = null; + if (iter.hasNext()) { + entry = iter.next(); + key = entry.get().getKey(); + value = entry.get().getValue(); + } + InOrder verifier = inOrder(rocksDBIteratorMock); + + verifier.verify(rocksDBIteratorMock, times(1)).isValid(); + verifier.verify(rocksDBIteratorMock, times(1)).key(); + assertArrayEquals(new byte[]{0x00}, key); + assertArrayEquals(new byte[]{0x7f}, value); } - - InOrder verifier = inOrder(rocksDBIteratorMock); - - verifier.verify(rocksDBIteratorMock, times(1)).isValid(); - verifier.verify(rocksDBIteratorMock, times(1)).key(); - assertArrayEquals(new byte[]{0x00}, key); - assertArrayEquals(new byte[]{0x7f}, value); } @Test @@ -261,14 +266,14 @@ public void testNullPrefixedIterator() throws IOException { RDBStoreByteArrayIterator iter = newIterator(null); verify(rocksDBIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksDBIteratorMock); - + when(rocksDBIteratorMock.isValid()).thenReturn(true); iter.seekToFirst(); + verify(rocksDBIteratorMock, times(1)).isValid(); + verify(rocksDBIteratorMock, times(1)).key(); verify(rocksDBIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksDBIteratorMock); - - when(rocksDBIteratorMock.isValid()).thenReturn(true); assertTrue(iter.hasNext()); - verify(rocksDBIteratorMock, times(1)).isValid(); + verify(rocksDBIteratorMock, times(0)).isValid(); verify(rocksDBIteratorMock, times(0)).key(); iter.seekToLast(); @@ -283,16 +288,17 @@ public void testNormalPrefixedIterator() throws IOException { RDBStoreByteArrayIterator iter = newIterator(testPrefix); verify(rocksDBIteratorMock, times(1)).seek(testPrefix); clearInvocations(rocksDBIteratorMock); - - iter.seekToFirst(); - verify(rocksDBIteratorMock, times(1)).seek(testPrefix); - clearInvocations(rocksDBIteratorMock); - when(rocksDBIteratorMock.isValid()).thenReturn(true); when(rocksDBIteratorMock.key()).thenReturn(testPrefix); - assertTrue(iter.hasNext()); + iter.seekToFirst(); verify(rocksDBIteratorMock, times(1)).isValid(); verify(rocksDBIteratorMock, times(1)).key(); + verify(rocksDBIteratorMock, times(1)).seek(testPrefix); + clearInvocations(rocksDBIteratorMock); + assertTrue(iter.hasNext()); + // hasNext shouldn't make isValid() redundant calls. + verify(rocksDBIteratorMock, times(0)).isValid(); + verify(rocksDBIteratorMock, times(0)).key(); Exception e = assertThrows(Exception.class, () -> iter.seekToLast(), "Prefixed iterator does not support seekToLast"); assertInstanceOf(UnsupportedOperationException.class, e); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 05a4c946394e..5b6c173d272d 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.utils.db; import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -37,14 +38,24 @@ import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksObjectUtils; import org.apache.log4j.Level; import org.apache.log4j.Logger; +import org.apache.ozone.test.GenericTestUtils; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.InOrder; import org.mockito.stubbing.Answer; import org.rocksdb.RocksIterator; @@ -73,12 +84,16 @@ ManagedRocksIterator newManagedRocksIterator() { } RDBStoreCodecBufferIterator newIterator() { - return new RDBStoreCodecBufferIterator(managedRocksIterator, null, null); + return newIterator(1); + } + + RDBStoreCodecBufferIterator newIterator(int numberOfThreads) { + return new RDBStoreCodecBufferIterator(managedRocksIterator, null, null, numberOfThreads); } RDBStoreCodecBufferIterator newIterator(CodecBuffer prefix) { return new RDBStoreCodecBufferIterator( - managedRocksIterator, rdbTableMock, prefix); + managedRocksIterator, rdbTableMock, prefix, 1); } Answer newAnswerInt(String name, int b) { @@ -90,38 +105,101 @@ Answer newAnswer(String name, byte... b) { System.out.printf("answer %s: %s%n", name, StringUtils.bytes2Hex(b)); Object[] args = invocation.getArguments(); final ByteBuffer buffer = (ByteBuffer) args[0]; - buffer.clear(); - buffer.put(b); - buffer.flip(); - return b.length; + return writeToBuffer(buffer, b); }; } + private int writeToBuffer(ByteBuffer buffer, byte... bytesToWrite) { + buffer.clear(); + buffer.put(bytesToWrite); + buffer.flip(); + return bytesToWrite.length; + } + + @ParameterizedTest + @ValueSource(ints = {0, 1, 2, 3, 5, 10}) + public void testRDBStoreCodecBufferIterGetsFailBeyondMaxBuffers(int maxBuffers) + throws InterruptedException, TimeoutException { + List>> vals = new ArrayList<>(); + when(rocksIteratorMock.isValid()).thenReturn(true); + AtomicInteger counter = new AtomicInteger(0); + + when(rocksIteratorMock.key(any())) + .thenAnswer(i -> writeToBuffer(i.getArgument(0), (byte)counter.getAndIncrement())); + when(rocksIteratorMock.value(any())) + .thenAnswer(i -> writeToBuffer(i.getArgument(0), (byte)counter.getAndIncrement())); + try (RDBStoreCodecBufferIterator iterator = newIterator(maxBuffers)) { + for (int i = 0; i < maxBuffers - 1; i++) { + vals.add(iterator.next()); + } + assertEquals(Math.max(maxBuffers - 1, 0), vals.size()); + ExecutorService executor = Executors.newSingleThreadExecutor(); + AtomicReference> nextThread = new AtomicReference<>(CompletableFuture.supplyAsync( + () -> { + UncheckedAutoCloseableSupplier> v = iterator.next(); + vals.add(v); + return true; + }, + executor)); + + if (maxBuffers <= 1) { + // Number of max buffers is always going to be 2. We need atleast 2 buffers one for getting the next value + // and one for returning the current value. + GenericTestUtils.waitFor(() -> nextThread.get().isDone() && nextThread.get().join(), 10, 100); + System.out.println(Thread.currentThread().getName()); + nextThread.set(CompletableFuture.supplyAsync( + () -> { + UncheckedAutoCloseableSupplier> v = iterator.next(); + vals.add(v); + return true; + }, + executor)); + } + assertEquals(Math.max(maxBuffers - 1, 1), vals.size()); + for (int i = 0; i < vals.size(); i++) { + assertEquals(2 * i, vals.get(i).get().getKey().getArray()[0]); + assertEquals(2 * i + 1, vals.get(i).get().getValue().getArray()[0]); + } + assertFalse(nextThread.get().isDone()); + int size = vals.size(); + vals.get(0).close(); + GenericTestUtils.waitFor(() -> nextThread.get().isDone() && nextThread.get().join(), 10, 100); + assertEquals(size + 1, vals.size()); + assertEquals(2 * size, vals.get(size).get().getKey().getArray()[0]); + assertEquals(2 * size + 1, vals.get(size).get().getValue().getArray()[0]); + for (UncheckedAutoCloseableSupplier> v : vals) { + v.close(); + } + executor.shutdown(); + } + } + @Test public void testForEachRemaining() throws Exception { when(rocksIteratorMock.isValid()) - .thenReturn(true, true, true, true, true, true, true, false); + .thenReturn(true, true, true, false); when(rocksIteratorMock.key(any())) .then(newAnswerInt("key1", 0x00)) - .then(newAnswerInt("key2", 0x00)) - .then(newAnswerInt("key3", 0x01)) - .then(newAnswerInt("key4", 0x02)) + .then(newAnswerInt("key2", 0x01)) + .then(newAnswerInt("key3", 0x02)) .thenThrow(new NoSuchElementException()); when(rocksIteratorMock.value(any())) .then(newAnswerInt("val1", 0x7f)) - .then(newAnswerInt("val2", 0x7f)) - .then(newAnswerInt("val3", 0x7e)) - .then(newAnswerInt("val4", 0x7d)) + .then(newAnswerInt("val2", 0x7e)) + .then(newAnswerInt("val3", 0x7d)) .thenThrow(new NoSuchElementException()); List> remaining = new ArrayList<>(); try (RDBStoreCodecBufferIterator i = newIterator()) { - i.forEachRemaining(kv -> { + i.forEachRemaining(kvSupplier -> { try { + Table.KeyValue kv = kvSupplier.get(); remaining.add(RawKeyValue.create( kv.getKey().getArray(), kv.getValue().getArray())); } catch (IOException e) { throw new RuntimeException(e); + } finally { + kvSupplier.close(); } }); @@ -138,11 +216,15 @@ public void testForEachRemaining() throws Exception { } @Test - public void testHasNextDependsOnIsvalid() throws Exception { - when(rocksIteratorMock.isValid()).thenReturn(true, true, false); + public void testHasNextDoesNotDependsOnIsvalid() throws Exception { + when(rocksIteratorMock.isValid()).thenReturn(true, false); try (RDBStoreCodecBufferIterator i = newIterator()) { assertTrue(i.hasNext()); + assertTrue(i.hasNext()); + i.next(); + assertFalse(i.hasNext()); + assertThrows(NoSuchElementException.class, i::next); assertFalse(i.hasNext()); } @@ -205,9 +287,8 @@ public void testSeekReturnsTheActualKey() throws Exception { .then(newAnswerInt("val1", 0x7f)); try (RDBStoreCodecBufferIterator i = newIterator(); - CodecBuffer target = CodecBuffer.wrap(new byte[]{0x55})) { - final Table.KeyValue val = i.seek(target); - + CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); + UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { InOrder verifier = inOrder(rocksIteratorMock); verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time @@ -217,8 +298,8 @@ public void testSeekReturnsTheActualKey() throws Exception { verifier.verify(rocksIteratorMock, times(1)).isValid(); verifier.verify(rocksIteratorMock, times(1)).key(any()); verifier.verify(rocksIteratorMock, times(1)).value(any()); - assertArrayEquals(new byte[]{0x00}, val.getKey().getArray()); - assertArrayEquals(new byte[]{0x7f}, val.getValue().getArray()); + assertArrayEquals(new byte[] {0x00}, valSupplier.get().getKey().getArray()); + assertArrayEquals(new byte[] {0x7f}, valSupplier.get().getValue().getArray()); } CodecTestUtil.gc(); @@ -233,7 +314,9 @@ public void testGettingTheKeyIfIteratorIsValid() throws Exception { byte[] key = null; try (RDBStoreCodecBufferIterator i = newIterator()) { if (i.hasNext()) { - key = i.next().getKey().getArray(); + try (UncheckedAutoCloseableSupplier> kv = i.next()) { + key = kv.get().getKey().getArray(); + } } } @@ -258,9 +341,10 @@ public void testGettingTheValueIfIteratorIsValid() throws Exception { byte[] value = null; try (RDBStoreCodecBufferIterator i = newIterator()) { if (i.hasNext()) { - Table.KeyValue entry = i.next(); - key = entry.getKey().getArray(); - value = entry.getValue().getArray(); + try (UncheckedAutoCloseableSupplier> entry = i.next()) { + key = entry.get().getKey().getArray(); + value = entry.get().getValue().getArray(); + } } } @@ -291,7 +375,6 @@ public void testRemovingFromDBActuallyDeletesFromTable() throws Exception { verifier.verify(rocksIteratorMock, times(1)).isValid(); verifier.verify(rdbTableMock, times(1)) .delete(ByteBuffer.wrap(testKey)); - CodecTestUtil.gc(); } @@ -316,17 +399,20 @@ public void testCloseCloses() throws Exception { @Test public void testNullPrefixedIterator() throws Exception { + try (RDBStoreCodecBufferIterator i = newIterator()) { verify(rocksIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksIteratorMock); - + when(rocksIteratorMock.isValid()).thenReturn(true); i.seekToFirst(); + verify(rocksIteratorMock, times(1)).isValid(); + verify(rocksIteratorMock, times(1)).key(any()); verify(rocksIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksIteratorMock); - when(rocksIteratorMock.isValid()).thenReturn(true); assertTrue(i.hasNext()); - verify(rocksIteratorMock, times(1)).isValid(); + // hasNext shouldn't make isValid() redundant calls. + verify(rocksIteratorMock, times(0)).isValid(); verify(rocksIteratorMock, times(0)).key(any()); i.seekToLast(); @@ -344,17 +430,19 @@ public void testNormalPrefixedIterator() throws Exception { final ByteBuffer prefix = ByteBuffer.wrap(prefixBytes); verify(rocksIteratorMock, times(1)).seek(prefix); clearInvocations(rocksIteratorMock); - - i.seekToFirst(); - verify(rocksIteratorMock, times(1)).seek(prefix); - clearInvocations(rocksIteratorMock); - when(rocksIteratorMock.isValid()).thenReturn(true); when(rocksIteratorMock.key(any())) .then(newAnswer("key1", prefixBytes)); - assertTrue(i.hasNext()); + i.seekToFirst(); verify(rocksIteratorMock, times(1)).isValid(); verify(rocksIteratorMock, times(1)).key(any()); + verify(rocksIteratorMock, times(1)).seek(prefix); + clearInvocations(rocksIteratorMock); + + assertTrue(i.hasNext()); + // Ensure redundant native call is made since key and value already have been fetched as part of seek + verify(rocksIteratorMock, times(0)).isValid(); + verify(rocksIteratorMock, times(0)).key(any()); Exception e = assertThrows(Exception.class, () -> i.seekToLast(), "Prefixed iterator does not support seekToLast"); From aab6ebc5d722b6ce4ed069aba087708e178022d1 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Mon, 14 Apr 2025 14:04:17 -0700 Subject: [PATCH 02/31] HDDS-12742. Remove log Change-Id: Ifdfafe0ec0a3467b012a0abb407bc8d5e75a88a3 --- .../hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index cb1fadcba6db..dabaa438320e 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -29,8 +29,6 @@ import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ReferenceCountedObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Implement {@link RDBStoreAbstractIterator} using {@link CodecBuffer}. @@ -38,7 +36,6 @@ */ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator { - private static final Logger LOG = LoggerFactory.getLogger(RDBStoreCodecBufferIterator.class); static class Buffer { private final CodecBuffer.Capacity initialCapacity; private final PutToByteBuffer source; From 5dd74606b332a06e84aa3661a083e7ed001502ba Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 15 Apr 2025 12:43:10 -0700 Subject: [PATCH 03/31] HDDS-12742. Remove synchronized Change-Id: I2b0ec8b91d0c9ac47b20b05665b5d31d8d486067 --- .../hdds/utils/db/RDBStoreAbstractIterator.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index 686e083904c7..4a1c396088aa 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -114,7 +114,7 @@ public final boolean hasNext() { } @Override - public final synchronized UncheckedAutoCloseableSupplier> next() { + public final UncheckedAutoCloseableSupplier> next() { if (hasNext()) { UncheckedAutoCloseableSupplier> entry = currentEntry.retainAndReleaseOnClose(); rocksDBIterator.get().next(); @@ -125,7 +125,7 @@ public final synchronized UncheckedAutoCloseableSupplier> next( } @Override - public final synchronized void seekToFirst() { + public final void seekToFirst() { if (prefix == null) { rocksDBIterator.get().seekToFirst(); } else { @@ -135,7 +135,7 @@ public final synchronized void seekToFirst() { } @Override - public final synchronized void seekToLast() { + public final void seekToLast() { if (prefix == null) { rocksDBIterator.get().seekToLast(); } else { @@ -145,7 +145,7 @@ public final synchronized void seekToLast() { } @Override - public final synchronized UncheckedAutoCloseableSupplier> seek(RAW key) { + public final UncheckedAutoCloseableSupplier> seek(RAW key) { seek0(key); setCurrentEntry(); // Current entry should be only closed when the next() and thus closing the returned entry should be a noop. @@ -153,7 +153,7 @@ public final synchronized UncheckedAutoCloseableSupplier> seek( } @Override - public final synchronized void removeFromDB() throws IOException { + public final void removeFromDB() throws IOException { if (rocksDBTable == null) { throw new UnsupportedOperationException("remove"); } @@ -165,7 +165,7 @@ public final synchronized void removeFromDB() throws IOException { } @Override - public synchronized void close() { + public void close() { rocksDBIterator.close(); closed = true; setCurrentEntry(); From e4bf8f78e624dde022473a7e9f09f110a54e341d Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 08:18:19 -0700 Subject: [PATCH 04/31] HDDS-12742. Fix seek Change-Id: I5bd5babb70126b2a59ab082e7c0c7469398817b2 --- .../utils/db/RDBStoreAbstractIterator.java | 18 ++++++++++++++++- .../db/TestRDBStoreByteArrayIterator.java | 19 ++++++++++++++++++ .../db/TestRDBStoreCodecBufferIterator.java | 20 +++++++++++++++++++ 3 files changed, 56 insertions(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index 4a1c396088aa..4d67085aec0c 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -37,6 +37,7 @@ abstract class RDBStoreAbstractIterator private static final Logger LOG = LoggerFactory.getLogger(RDBStoreAbstractIterator.class); + private final UncheckedAutoCloseableSupplier> empty; private final ManagedRocksIterator rocksDBIterator; private final RDBTable rocksDBTable; private ReferenceCountedObject> currentEntry; @@ -54,6 +55,18 @@ abstract class RDBStoreAbstractIterator this.currentEntry = null; this.hasNext = false; this.closed = false; + this.empty = new UncheckedAutoCloseableSupplier>() { + + @Override + public void close() { + + } + + @Override + public RawKeyValue get() { + return null; + } + }; } /** @return the {@link Table.KeyValue} for the current entry. */ @@ -149,7 +162,10 @@ public final UncheckedAutoCloseableSupplier> seek(RAW key) { seek0(key); setCurrentEntry(); // Current entry should be only closed when the next() and thus closing the returned entry should be a noop. - return currentEntry.retainAndReleaseOnClose(); + if (hasNext()) { + return currentEntry.retainAndReleaseOnClose(); + } + return empty; } @Override diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index 6b6674f87fee..770b7f9b7c55 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -20,6 +20,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentCaptor.forClass; @@ -166,6 +167,24 @@ public void testSeekToLastSeeks() { verify(rocksDBIteratorMock, times(1)).seekToLast(); } + @Test + public void testSeekWithInvalidValue() { + when(rocksDBIteratorMock.isValid()).thenReturn(false); + + try (RDBStoreByteArrayIterator iter = newIterator()) { + final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[] {0x55}); + InOrder verifier = inOrder(rocksDBIteratorMock); + + verify(rocksDBIteratorMock, times(1)).seekToFirst(); //at construct time + verify(rocksDBIteratorMock, never()).seekToLast(); + verifier.verify(rocksDBIteratorMock, times(1)).seek(any(byte[].class)); + verifier.verify(rocksDBIteratorMock, times(1)).isValid(); + verifier.verify(rocksDBIteratorMock, never()).key(); + verifier.verify(rocksDBIteratorMock, never()).value(); + assertNull(val.get()); + } + } + @Test public void testSeekReturnsTheActualKey() throws Exception { when(rocksDBIteratorMock.isValid()).thenReturn(true); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 5b6c173d272d..052fa7901cf1 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -21,6 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; @@ -278,6 +279,25 @@ public void testSeekToLastSeeks() throws Exception { CodecTestUtil.gc(); } + @Test + public void testSeekWithInvalidValue() { + when(rocksIteratorMock.isValid()).thenReturn(false); + + try (RDBStoreCodecBufferIterator i = newIterator(); + CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); + UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { + InOrder verifier = inOrder(rocksIteratorMock); + verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time + verify(rocksIteratorMock, never()).seekToLast(); + verifier.verify(rocksIteratorMock, times(1)) + .seek(any(ByteBuffer.class)); + verifier.verify(rocksIteratorMock, times(1)).isValid(); + verifier.verify(rocksIteratorMock, never()).key(any()); + verifier.verify(rocksIteratorMock, never()).value(any()); + assertNull(valSupplier.get()); + } + } + @Test public void testSeekReturnsTheActualKey() throws Exception { when(rocksIteratorMock.isValid()).thenReturn(true); From 9ab91c91addbb0486c11d3bb31ff3fc588ce255a Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 08:20:24 -0700 Subject: [PATCH 05/31] HDDS-12742. Add test case Change-Id: Id34d08479c4baedc28f177e82ed9707aff481da8 --- .../hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java | 4 ++-- .../hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index 770b7f9b7c55..f6d768cb43b3 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -173,8 +173,8 @@ public void testSeekWithInvalidValue() { try (RDBStoreByteArrayIterator iter = newIterator()) { final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[] {0x55}); + assertFalse(iter.hasNext()); InOrder verifier = inOrder(rocksDBIteratorMock); - verify(rocksDBIteratorMock, times(1)).seekToFirst(); //at construct time verify(rocksDBIteratorMock, never()).seekToLast(); verifier.verify(rocksDBIteratorMock, times(1)).seek(any(byte[].class)); @@ -193,7 +193,7 @@ public void testSeekReturnsTheActualKey() throws Exception { RDBStoreByteArrayIterator iter = newIterator(); final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[]{0x55}); - + assertTrue(iter.hasNext()); InOrder verifier = inOrder(rocksDBIteratorMock); verify(rocksDBIteratorMock, times(1)).seekToFirst(); //at construct time diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 052fa7901cf1..4084697cecb5 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -286,6 +286,7 @@ public void testSeekWithInvalidValue() { try (RDBStoreCodecBufferIterator i = newIterator(); CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { + assertFalse(i.hasNext()); InOrder verifier = inOrder(rocksIteratorMock); verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time verify(rocksIteratorMock, never()).seekToLast(); @@ -309,8 +310,8 @@ public void testSeekReturnsTheActualKey() throws Exception { try (RDBStoreCodecBufferIterator i = newIterator(); CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { + assertTrue(i.hasNext()); InOrder verifier = inOrder(rocksIteratorMock); - verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time verify(rocksIteratorMock, never()).seekToLast(); verifier.verify(rocksIteratorMock, times(1)) From abe06abb83a5b658ebd22efb70bb03a12483918f Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 08:55:40 -0700 Subject: [PATCH 06/31] HDDS-12742. Fix iter Change-Id: I54f7ebeb3967947131ae7acfefa17990965f5ca4 --- .../main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index cbd4385e98bd..38f63cb3a9bc 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -669,7 +669,7 @@ public void seekToLast() { public KeyValue seek(KEY key) throws IOException { try (AutoCloseSupplier rawKey = convert(key); UncheckedAutoCloseableSupplier> result = rawIterator.seek(rawKey.get())) { - return result == null ? null : convert(result.get()); + return result.get() == null ? null : convert(result.get()); } } From d215d4ba219cbd73fcbef57f66abeb094b5dce25 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 09:06:29 -0700 Subject: [PATCH 07/31] HDDS-12742. Fix NPE Change-Id: Ic0e600dfa46951e048231a4e56a20bd9ce6fa423 --- .../hdds/utils/db/RDBStoreAbstractIterator.java | 15 +-------------- .../org/apache/hadoop/hdds/utils/db/RDBTable.java | 5 ++--- .../utils/db/TestRDBStoreByteArrayIterator.java | 2 +- .../utils/db/TestRDBStoreCodecBufferIterator.java | 2 +- 4 files changed, 5 insertions(+), 19 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index 4d67085aec0c..1935faef3ebe 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -37,7 +37,6 @@ abstract class RDBStoreAbstractIterator private static final Logger LOG = LoggerFactory.getLogger(RDBStoreAbstractIterator.class); - private final UncheckedAutoCloseableSupplier> empty; private final ManagedRocksIterator rocksDBIterator; private final RDBTable rocksDBTable; private ReferenceCountedObject> currentEntry; @@ -55,18 +54,6 @@ abstract class RDBStoreAbstractIterator this.currentEntry = null; this.hasNext = false; this.closed = false; - this.empty = new UncheckedAutoCloseableSupplier>() { - - @Override - public void close() { - - } - - @Override - public RawKeyValue get() { - return null; - } - }; } /** @return the {@link Table.KeyValue} for the current entry. */ @@ -165,7 +152,7 @@ public final UncheckedAutoCloseableSupplier> seek(RAW key) { if (hasNext()) { return currentEntry.retainAndReleaseOnClose(); } - return empty; + return null; } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index 4ee9755a4223..0c57b897b150 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -222,8 +222,7 @@ public TableIterator> iterator() @Override public TableIterator> iterator(byte[] prefix) throws IOException { - TableIterator>> itr = - new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); + RDBStoreByteArrayIterator itr = new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); return new TableIterator>() { @Override public void seekToFirst() { @@ -238,7 +237,7 @@ public void seekToLast() { @Override public KeyValue seek(byte[] bytes) throws IOException { try (UncheckedAutoCloseableSupplier> kv = itr.seek(bytes)) { - return kv.get(); + return kv == null ? null : kv.get(); } } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index f6d768cb43b3..3dc57e759494 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -181,7 +181,7 @@ public void testSeekWithInvalidValue() { verifier.verify(rocksDBIteratorMock, times(1)).isValid(); verifier.verify(rocksDBIteratorMock, never()).key(); verifier.verify(rocksDBIteratorMock, never()).value(); - assertNull(val.get()); + assertNull(val); } } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 4084697cecb5..88661aaaf00b 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -295,7 +295,7 @@ public void testSeekWithInvalidValue() { verifier.verify(rocksIteratorMock, times(1)).isValid(); verifier.verify(rocksIteratorMock, never()).key(any()); verifier.verify(rocksIteratorMock, never()).value(any()); - assertNull(valSupplier.get()); + assertNull(valSupplier); } } From 85c74b47cd2848b5a292c7810c4872c798076500 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 14:01:54 -0700 Subject: [PATCH 08/31] HDDS-12742. Fix NPE Change-Id: Ia271b1d6be0a4a526a4a65fda7d1ad6dc9cd2bd2 --- .../main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 38f63cb3a9bc..cbd4385e98bd 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -669,7 +669,7 @@ public void seekToLast() { public KeyValue seek(KEY key) throws IOException { try (AutoCloseSupplier rawKey = convert(key); UncheckedAutoCloseableSupplier> result = rawIterator.seek(rawKey.get())) { - return result.get() == null ? null : convert(result.get()); + return result == null ? null : convert(result.get()); } } From 7d82a9eae5d5717c5cb1e0f10265eccfb82cfd82 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 16:45:06 -0700 Subject: [PATCH 09/31] HDDS-12742. Add Blocking Deque instead custom implementation of a blcking stack Change-Id: I78272536972d58b76a10092bfde418a29ba8adae --- .../utils/db/RDBStoreCodecBufferIterator.java | 87 ++++++++++--------- 1 file changed, 45 insertions(+), 42 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index dabaa438320e..58fb85942f48 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -18,12 +18,14 @@ package org.apache.hadoop.hdds.utils.db; import java.io.IOException; +import java.util.Deque; 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.concurrent.BlockingDeque; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import org.apache.commons.lang3.exception.UncheckedInterruptedException; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; @@ -92,10 +94,9 @@ CodecBuffer getFromDb() { } } - private final Stack> availableBufferStack; + private final BlockingDeque> availableBufferStack; private final Set> inUseBuffers; - private final Object bufferLock; - private final AtomicBoolean closed = new AtomicBoolean(); + private final AtomicReference closed = new AtomicReference<>(false); RDBStoreCodecBufferIterator(ManagedRocksIterator iterator, RDBTable table, CodecBuffer prefix, int maxNumberOfBuffersInMemory) { @@ -103,7 +104,7 @@ CodecBuffer getFromDb() { // 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.availableBufferStack = new Stack<>(); + this.availableBufferStack = new LinkedBlockingDeque<>(maxNumberOfBuffersInMemory); this.inUseBuffers = new HashSet<>(); for (int i = 0; i < maxNumberOfBuffersInMemory; i++) { Buffer keyBuffer = new Buffer( @@ -112,10 +113,9 @@ CodecBuffer getFromDb() { Buffer valueBuffer = new Buffer( new CodecBuffer.Capacity(name + "-iterator-value-" + i, 4 << 10), buffer -> getRocksDBIterator().get().value(buffer)); - availableBufferStack.push(new RawKeyValue<>(keyBuffer, valueBuffer)); + availableBufferStack.add(new RawKeyValue<>(keyBuffer, valueBuffer)); } - this.bufferLock = new Object(); seekToFirst(); } @@ -123,46 +123,51 @@ void assertOpen() { Preconditions.assertTrue(!closed.get(), "Already closed"); } - private V getFromStack(Object lock, Stack stack, Set inUseSet) { - synchronized (Objects.requireNonNull(lock)) { - while (stack.isEmpty()) { - try { - assertOpen(); - lock.wait(1000); - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } + private V getFromDeque(BlockingDeque deque, Set inUseSet) { + V popped; + do { + try { + popped = deque.poll(1000, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); } - V popped = stack.pop(); - inUseSet.add(popped); - return popped; - } + } while(popped == null); + assertOpen(); + inUseSet.add(popped); + return popped; } private ReferenceCountedObject> getReferenceCountedBuffer( - RawKeyValue key, Stack> stack, Set> inUseSet, - Object lock, Function, RawKeyValue> transformer) { + RawKeyValue key, Deque> stack, Set> inUseSet, + Function, RawKeyValue> transformer) { RawKeyValue value = transformer.apply(key); return ReferenceCountedObject.wrap(value, () -> { }, completelyReleased -> { if (!completelyReleased) { return; } - synchronized (lock) { - stack.push(key); - inUseSet.remove(key); - lock.notify(); - } + closed.updateAndGet((prev) -> { + // If already closed the data structure should not be manipulated since the buffer would have already been + // closed. + if (!prev) { + //Entire block done inside this code block to avoid race condition with close() method. + //Remove from the set before adding it back to the stack. Otherwise there could be a race condition with + // #getFromDeque function. + inUseSet.remove(key); + stack.push(key); + } + return prev; + }); }); } @Override ReferenceCountedObject> getKeyValue() { assertOpen(); - RawKeyValue kvBuffer = getFromStack(bufferLock, availableBufferStack, inUseBuffers); + RawKeyValue kvBuffer = getFromDeque(availableBufferStack, inUseBuffers); Function, RawKeyValue> transformer = kv -> new RawKeyValue<>(kv.getKey().getFromDb(), kv.getValue().getFromDb()); - return getReferenceCountedBuffer(kvBuffer, availableBufferStack, inUseBuffers, bufferLock, transformer); + return getReferenceCountedBuffer(kvBuffer, availableBufferStack, inUseBuffers, transformer); } @Override @@ -190,25 +195,23 @@ boolean startsWithPrefix(CodecBuffer key) { return key.startsWith(prefix); } - private void release(Stack valueStack, Set inUseSet, Object lock, Function releaser) { - synchronized (Objects.requireNonNull(lock)) { - while (!valueStack.isEmpty()) { - V popped = valueStack.pop(); - releaser.apply(popped); - } + private void release(Deque valueStack, Set inUseSet, Function releaser) { + while (!valueStack.isEmpty()) { + V popped = valueStack.pop(); + releaser.apply(popped); + } - for (V inUseValue : inUseSet) { - releaser.apply(inUseValue); - } + for (V inUseValue : inUseSet) { + releaser.apply(inUseValue); } } @Override - public synchronized void close() { + public void close() { if (closed.compareAndSet(false, true)) { super.close(); Optional.ofNullable(getPrefix()).ifPresent(CodecBuffer::release); - release(availableBufferStack, inUseBuffers, bufferLock, kv -> { + release(availableBufferStack, inUseBuffers, kv -> { kv.getKey().release(); kv.getValue().release(); return null; From ce6ab81878e1fdd900a4ad1e4ec5631418a47bf7 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 16:50:05 -0700 Subject: [PATCH 10/31] HDDS-12742. Fix checkstyle Change-Id: I95dcc0d6e370329846272441dc70f4be2b671a29 --- .../hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 58fb85942f48..06aba1589ef4 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -131,7 +131,7 @@ private V getFromDeque(BlockingDeque deque, Set inUseSet) { } catch (InterruptedException e) { throw new UncheckedInterruptedException(e); } - } while(popped == null); + } while (popped == null); assertOpen(); inUseSet.add(popped); return popped; @@ -200,7 +200,6 @@ private void release(Deque valueStack, Set inUseSet, Function V popped = valueStack.pop(); releaser.apply(popped); } - for (V inUseValue : inUseSet) { releaser.apply(inUseValue); } From 091e8ca58164dcb9c7a17b6cdde768217a39cf2f Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 16 Apr 2025 18:18:13 -0700 Subject: [PATCH 11/31] HDDS-12742. Fix test cases Change-Id: I4db792cbf85d60f371653e7b9119d5155661c228 --- .../hadoop/hdds/utils/db/RDBStoreAbstractIterator.java | 7 +++++-- .../java/org/apache/hadoop/hdds/utils/db/RDBTable.java | 8 ++++---- .../hdds/utils/db/TestRDBStoreByteArrayIterator.java | 1 + .../hdds/utils/db/TestRDBStoreCodecBufferIterator.java | 1 + .../apache/hadoop/hdds/utils/db/TestRDBTableStore.java | 3 +++ 5 files changed, 14 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index 1935faef3ebe..e3f4d8603c3b 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -40,6 +40,7 @@ abstract class RDBStoreAbstractIterator private final ManagedRocksIterator rocksDBIterator; private final RDBTable rocksDBTable; private ReferenceCountedObject> currentEntry; + private RawKeyValue previousKeyValue; // This is for schemas that use a fixed-length // prefix for each key. private final RAW prefix; @@ -54,6 +55,7 @@ abstract class RDBStoreAbstractIterator this.currentEntry = null; this.hasNext = false; this.closed = false; + this.previousKeyValue = null; } /** @return the {@link Table.KeyValue} for the current entry. */ @@ -117,6 +119,7 @@ public final boolean hasNext() { public final UncheckedAutoCloseableSupplier> next() { if (hasNext()) { UncheckedAutoCloseableSupplier> entry = currentEntry.retainAndReleaseOnClose(); + this.previousKeyValue = entry.get(); rocksDBIterator.get().next(); setCurrentEntry(); return entry; @@ -160,8 +163,8 @@ public final void removeFromDB() throws IOException { if (rocksDBTable == null) { throw new UnsupportedOperationException("remove"); } - if (currentEntry.get() != null) { - delete(currentEntry.get().getKey()); + if (previousKeyValue != null) { + delete(previousKeyValue.getKey()); } else { LOG.info("Failed to removeFromDB: currentEntry == null"); } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index 0c57b897b150..67d5f6f75621 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -235,19 +235,19 @@ public void seekToLast() { } @Override - public KeyValue seek(byte[] bytes) throws IOException { + public KeyValue seek(byte[] bytes) { try (UncheckedAutoCloseableSupplier> kv = itr.seek(bytes)) { return kv == null ? null : kv.get(); } } @Override - public void removeFromDB() { - itr.remove(); + public void removeFromDB() throws IOException { + itr.removeFromDB(); } @Override - public void close() throws IOException { + public void close() { itr.close(); } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index 3dc57e759494..d319d15fb3ed 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -257,6 +257,7 @@ public void testRemovingFromDBActuallyDeletesFromTable() throws Exception { when(rocksDBIteratorMock.key()).thenReturn(testKey); RDBStoreByteArrayIterator iter = newIterator(null); + iter.next(); iter.removeFromDB(); InOrder verifier = inOrder(rocksDBIteratorMock, rocksTableMock); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 88661aaaf00b..489052240039 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -388,6 +388,7 @@ public void testRemovingFromDBActuallyDeletesFromTable() throws Exception { .then(newAnswer("key1", testKey)); try (RDBStoreCodecBufferIterator i = newIterator(null)) { + i.next(); i.removeFromDB(); } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java index 37f81369f91f..06a0f0f189c3 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBTableStore.java @@ -495,6 +495,7 @@ public void testIteratorRemoveFromDB() throws Exception { writeToTable(testTable, 3); try (TableIterator> iterator = testTable.iterator()) { + iterator.next(); iterator.removeFromDB(); } assertNull(testTable.get(bytesOf[1])); @@ -508,6 +509,7 @@ public void testIteratorRemoveFromDB() throws Exception { try (TableIterator> iterator = testTable.iterator()) { iterator.seekToLast(); + iterator.next(); iterator.removeFromDB(); } assertNotNull(testTable.get(bytesOf[1])); @@ -521,6 +523,7 @@ public void testIteratorRemoveFromDB() throws Exception { try (TableIterator> iterator = testTable.iterator()) { iterator.seek(bytesOf[3]); + iterator.next(); iterator.removeFromDB(); } assertNotNull(testTable.get(bytesOf[1])); From 7fdced291a909204966af84447e21e99b4bca479 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 4 May 2025 10:36:16 -0400 Subject: [PATCH 12/31] HDDS-12742. Add Spliterator Change-Id: I73c8ff72786dcc57650bbd1107915946fc291396 --- .../hdds/utils/db/KeyValueSpliterator.java | 25 +++ .../apache/hadoop/hdds/utils/db/RDBTable.java | 7 +- .../apache/hadoop/hdds/utils/db/Table.java | 4 + .../hadoop/hdds/utils/db/TypedTable.java | 171 +++++++++++++++++- 4 files changed, 205 insertions(+), 2 deletions(-) create mode 100644 hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java new file mode 100644 index 000000000000..3f435b2ff130 --- /dev/null +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.utils.db; + +import java.io.Closeable; +import java.util.Spliterator; + +interface TableSpliterator extends Spliterator, Closeable { + +} diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index 5fc4bb479f08..c9f7f6dce42d 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -269,8 +269,13 @@ public TableIterator> TableIterator>> iterator( CodecBuffer prefix) throws IOException { + return iterator(prefix, 2); + } + + TableIterator>> iterator( + CodecBuffer prefix, int maxNumberOfBuffers) throws IOException { return new RDBStoreCodecBufferIterator(db.newIterator(family, false), - this, prefix, 2); + this, prefix, maxNumberOfBuffers); } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java index 18f98a08ed63..455c90661c38 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java @@ -420,4 +420,8 @@ public int hashCode() { interface KeyValueIterator extends TableIterator> { } + + /** A {@link org.apache.hadoop.hdds.utils.db.TableSpliterator} to split iterate {@link KeyValue}s. */ + interface KeyValueSpliterator extends TableSpliterator> { + } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 24629a989c87..c0e6021dba8e 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -30,6 +30,12 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Spliterator; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.TableCacheMetrics; @@ -42,6 +48,7 @@ import org.apache.hadoop.hdds.utils.db.cache.TableCache.CacheType; import org.apache.hadoop.hdds.utils.db.cache.TableNoCache; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.function.CheckedBiFunction; import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; @@ -555,7 +562,7 @@ public VALUE getValue() throws IOException { } } - RawIterator newCodecBufferTableIterator( + private RawIterator newCodecBufferTableIterator( TableIterator>> i) { return new RawIterator(i) { @Override @@ -585,6 +592,67 @@ KeyValue convert(RawKeyValue raw) }; } + private RawSpliterator newCodecBufferSpliterator(KEY prefix, KEY startKey, int maxParallelism) + throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism) { + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().readableBytes(); + final KEY key = keyCodec.fromCodecBuffer(kv.getKey()); + final VALUE value = valueCodec.fromCodecBuffer(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + TableIterator>> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + + CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); + CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); + try { + TableIterator>> itr = + rawTable.iterator(prefixBuffer, maxParallelism + 1); + if (startKeyBuffer != null) { + itr.seek(startKeyBuffer); + } + return itr; + } finally { + if (startKeyBuffer != null) { + startKeyBuffer.release(); + } + } + } + }; + } + + private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism) + throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism) { + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().length; + final KEY key = keyCodec.fromPersistedFormat(kv.getKey()); + final VALUE value = valueCodec.fromPersistedFormat(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + TableIterator>> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + byte[] prefixBytes = encodeKey(prefix); + byte[] startKeyBytes = encodeKey(startKey); + TableIterator>> itr = + rawTable.closeableSupplierIterator(prefixBytes); + if (startKeyBytes != null) { + itr.seek(startKeyBytes); + } + return itr; + } + }; + } + /** * Table Iterator implementation for strongly typed tables. */ @@ -671,4 +739,105 @@ public void removeFromDB() throws IOException { rawIterator.removeFromDB(); } } + + /** + * A {@link Table.KeyValueIterator} backed by a raw iterator. + * + * @param The raw type. + */ + + abstract class RawSpliterator implements KeyValueSpliterator { + + private final ReferenceCountedObject>>> rawIterator; + private final AtomicInteger maxNumberOfAdditionalSplits; + private final Lock lock; + private final AtomicReference closeException = new AtomicReference<>(); + + abstract KeyValue convert(RawKeyValue kv) throws IOException; + + abstract TableIterator>> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException; + + private RawSpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { + TableIterator>> itr = getRawIterator(prefix, + startKey, maxParallelism); + this.lock = new ReentrantLock(); + this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); + this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, + (completelyReleased) -> { + if (completelyReleased) { + lock.lock(); + try { + itr.close(); + } catch (IOException e) { + closeException.set(e); + } finally { + lock.unlock(); + } + } + this.maxNumberOfAdditionalSplits.incrementAndGet(); + }); + this.rawIterator.retain(); + } + + @Override + public boolean tryAdvance(Consumer> action) { + lock.lock(); + UncheckedAutoCloseableSupplier> kv = null; + try { + if (this.rawIterator.get().hasNext()) { + kv = rawIterator.get().next(); + } + } finally { + lock.unlock(); + } + try { + if (kv != null) { + action.accept(convert(kv.get())); + return true; + } + } catch (IOException e) { + throw new IllegalStateException("Failed next()", e); + } finally { + if (kv != null) { + kv.close(); + } + } + return false; + } + + @Override + public int characteristics() { + return Spliterator.DISTINCT; + } + + @Override + public Spliterator> trySplit() { + int val = maxNumberOfAdditionalSplits.decrementAndGet(); + if (val >= 1) { + try { + this.rawIterator.retain(); + } catch (Exception e) { + maxNumberOfAdditionalSplits.incrementAndGet(); + return null; + } + return this; + } + return null; + } + + @Override + public void close() throws IOException { + this.rawIterator.release(); + if (closeException.get() != null) { + throw closeException.get(); + } + } + + @Override + public long estimateSize() { + return Long.MAX_VALUE; + } + } } From f5b633bcb81a13265462258ce4e2b57e3ec6a032 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 4 May 2025 22:29:38 -0400 Subject: [PATCH 13/31] HDDS-12742. Fix Spliterator Change-Id: I46afdf6e6510fece9bca9a0d677e52b33dfc97c5 --- .../hadoop/hdds/utils/db/RawSpliterator.java | 128 ++++++++++++++++++ ...Spliterator.java => TableSpliterator.java} | 0 .../hadoop/hdds/utils/db/TypedTable.java | 118 +--------------- 3 files changed, 133 insertions(+), 113 deletions(-) create mode 100644 hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java rename hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/{KeyValueSpliterator.java => TableSpliterator.java} (100%) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java new file mode 100644 index 000000000000..f908c153f05b --- /dev/null +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.utils.db; + +import java.io.IOException; +import java.util.Spliterator; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; +import org.apache.ratis.util.ReferenceCountedObject; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; + +/** + * A {@link Table.KeyValueIterator} backed by a raw iterator. + * + * @param The raw type. + */ +abstract class RawSpliterator implements Table.KeyValueSpliterator { + + private final ReferenceCountedObject>>> rawIterator; + private final AtomicInteger maxNumberOfAdditionalSplits; + private final Lock lock; + private final AtomicReference closeException = new AtomicReference<>(); + + abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; + + abstract TableIterator>> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException; + + RawSpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { + TableIterator>> itr = getRawIterator(prefix, + startKey, maxParallelism); + this.lock = new ReentrantLock(); + this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); + this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, + (completelyReleased) -> { + if (completelyReleased) { + lock.lock(); + try { + itr.close(); + } catch (IOException e) { + closeException.set(e); + } finally { + lock.unlock(); + } + } + this.maxNumberOfAdditionalSplits.incrementAndGet(); + }); + this.rawIterator.retain(); + } + + @Override + public boolean tryAdvance(Consumer> action) { + lock.lock(); + UncheckedAutoCloseableSupplier> kv = null; + try { + if (this.rawIterator.get().hasNext()) { + kv = rawIterator.get().next(); + } + } finally { + lock.unlock(); + } + try { + if (kv != null) { + action.accept(convert(kv.get())); + return true; + } + } catch (IOException e) { + throw new IllegalStateException("Failed next()", e); + } finally { + if (kv != null) { + kv.close(); + } + } + return false; + } + + @Override + public int characteristics() { + return Spliterator.DISTINCT; + } + + @Override + public Spliterator> trySplit() { + int val = maxNumberOfAdditionalSplits.decrementAndGet(); + if (val >= 1) { + try { + this.rawIterator.retain(); + } catch (Exception e) { + maxNumberOfAdditionalSplits.incrementAndGet(); + return null; + } + return this; + } + return null; + } + + @Override + public void close() throws IOException { + this.rawIterator.release(); + if (closeException.get() != null) { + throw closeException.get(); + } + } + + @Override + public long estimateSize() { + return Long.MAX_VALUE; + } +} diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java similarity index 100% rename from hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/KeyValueSpliterator.java rename to hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index c0e6021dba8e..acdbb6ed072a 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -30,12 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Spliterator; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Consumer; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.TableCacheMetrics; @@ -48,7 +42,6 @@ import org.apache.hadoop.hdds.utils.db.cache.TableCache.CacheType; import org.apache.hadoop.hdds.utils.db.cache.TableNoCache; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.function.CheckedBiFunction; import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; @@ -592,9 +585,9 @@ KeyValue convert(RawKeyValue raw) }; } - private RawSpliterator newCodecBufferSpliterator(KEY prefix, KEY startKey, int maxParallelism) - throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism) { + private RawSpliterator newCodecBufferSpliterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism) { @Override KeyValue convert(RawKeyValue kv) throws IOException { @@ -626,9 +619,9 @@ TableIterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism) + private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism) { + return new RawSpliterator(prefix, startKey, maxParallelism) { @Override KeyValue convert(RawKeyValue kv) throws IOException { @@ -739,105 +732,4 @@ public void removeFromDB() throws IOException { rawIterator.removeFromDB(); } } - - /** - * A {@link Table.KeyValueIterator} backed by a raw iterator. - * - * @param The raw type. - */ - - abstract class RawSpliterator implements KeyValueSpliterator { - - private final ReferenceCountedObject>>> rawIterator; - private final AtomicInteger maxNumberOfAdditionalSplits; - private final Lock lock; - private final AtomicReference closeException = new AtomicReference<>(); - - abstract KeyValue convert(RawKeyValue kv) throws IOException; - - abstract TableIterator>> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException; - - private RawSpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { - TableIterator>> itr = getRawIterator(prefix, - startKey, maxParallelism); - this.lock = new ReentrantLock(); - this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); - this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, - (completelyReleased) -> { - if (completelyReleased) { - lock.lock(); - try { - itr.close(); - } catch (IOException e) { - closeException.set(e); - } finally { - lock.unlock(); - } - } - this.maxNumberOfAdditionalSplits.incrementAndGet(); - }); - this.rawIterator.retain(); - } - - @Override - public boolean tryAdvance(Consumer> action) { - lock.lock(); - UncheckedAutoCloseableSupplier> kv = null; - try { - if (this.rawIterator.get().hasNext()) { - kv = rawIterator.get().next(); - } - } finally { - lock.unlock(); - } - try { - if (kv != null) { - action.accept(convert(kv.get())); - return true; - } - } catch (IOException e) { - throw new IllegalStateException("Failed next()", e); - } finally { - if (kv != null) { - kv.close(); - } - } - return false; - } - - @Override - public int characteristics() { - return Spliterator.DISTINCT; - } - - @Override - public Spliterator> trySplit() { - int val = maxNumberOfAdditionalSplits.decrementAndGet(); - if (val >= 1) { - try { - this.rawIterator.retain(); - } catch (Exception e) { - maxNumberOfAdditionalSplits.incrementAndGet(); - return null; - } - return this; - } - return null; - } - - @Override - public void close() throws IOException { - this.rawIterator.release(); - if (closeException.get() != null) { - throw closeException.get(); - } - } - - @Override - public long estimateSize() { - return Long.MAX_VALUE; - } - } } From 1971a961efef76bfaae2a7782789f2d605cd2f02 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran <47532440+swamirishi@users.noreply.github.com> Date: Mon, 5 May 2025 14:32:20 -0400 Subject: [PATCH 14/31] Update hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java Co-authored-by: Ritesh H Shukla --- .../hdds/utils/db/RDBStoreCodecBufferIterator.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 6571efe7ade6..e849195048b0 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -33,9 +33,17 @@ 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. - */ + * An implementation of {@link RDBStoreAbstractIterator} that uses {@link CodecBuffer} + * for efficient memory management when iterating over RocksDB entries. + *

+ * This iterator employs a buffer pooling strategy to minimize memory allocations + * during iteration. Key and value buffers are pre-allocated and reused through a + * reference-counting mechanism, which significantly reduces GC pressure when + * processing large datasets. + *

+ * Key features: + *

    + **/ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator { private final BlockingDeque> availableBufferStack; From a33f2655f912423c746a71924af3709b19a69139 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 6 May 2025 12:39:12 -0400 Subject: [PATCH 15/31] HDDS-12742. Make concurrent hash set Change-Id: Ifdff95b06c134ff3a2d2bf7022edcf7c630c86c2 --- .../hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 6571efe7ade6..4a7e65922e29 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -29,6 +29,7 @@ import java.util.function.Function; import org.apache.commons.lang3.exception.UncheckedInterruptedException; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; +import org.apache.hadoop.util.Sets; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ReferenceCountedObject; @@ -49,7 +50,7 @@ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator maxNumberOfBuffersInMemory = Math.max(2, maxNumberOfBuffersInMemory); final String name = table != null ? table.getName() : null; this.availableBufferStack = new LinkedBlockingDeque<>(maxNumberOfBuffersInMemory); - this.inUseBuffers = new HashSet<>(); + this.inUseBuffers = Sets.newConcurrentHashSet(); for (int i = 0; i < maxNumberOfBuffersInMemory; i++) { Buffer keyBuffer = new Buffer( new CodecBuffer.Capacity(name + "-iterator-key-" + i, 1 << 10), From 71f9c28bc861e285bbfd1cfdaf8ced74dd29943d Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 6 May 2025 12:41:36 -0400 Subject: [PATCH 16/31] HDDS-12742. Fix checkstyle Change-Id: I9ea4eaf4b9f4db8c279a90d5222cb448d46772f6 --- .../apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 1891ff625fcb..8d80ddd9c248 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.Deque; -import java.util.HashSet; import java.util.Optional; import java.util.Set; import java.util.concurrent.BlockingDeque; From 5734027e1274f2f0ed53b8d722f5b9ec8c691328 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 6 May 2025 13:26:44 -0400 Subject: [PATCH 17/31] HDDS-12742. Fix pmd Change-Id: I819e2a82cf30d5851762ca2fe8c881c716035df4 --- .../utils/db/RDBStoreCodecBufferIterator.java | 3 --- .../apache/hadoop/hdds/utils/db/TypedTable.java | 16 ++++++++++++++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 8d80ddd9c248..4ce61d43a91f 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -35,14 +35,11 @@ /** * An implementation of {@link RDBStoreAbstractIterator} that uses {@link CodecBuffer} * for efficient memory management when iterating over RocksDB entries. - *

    * This iterator employs a buffer pooling strategy to minimize memory allocations * during iteration. Key and value buffers are pre-allocated and reused through a * reference-counting mechanism, which significantly reduces GC pressure when * processing large datasets. - *

    * Key features: - *

      **/ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index acdbb6ed072a..7a6312647ebe 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -400,6 +400,18 @@ public Table.KeyValueIterator iterator() throws IOException { return iterator(null); } + public Table.KeyValueSpliterator spliterator(int maxParallelism) throws IOException { + return spliterator(null, null, maxParallelism); + } + + public Table.KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism) + throws IOException { + if (supportCodecBuffer) { + return newCodecBufferSpliterator(prefix, startKey, maxParallelism); + } + return newByteArraySpliterator(prefix, startKey, maxParallelism); + } + @Override public Table.KeyValueIterator iterator(KEY prefix) throws IOException { @@ -610,6 +622,10 @@ TableIterator Date: Tue, 6 May 2025 13:29:24 -0400 Subject: [PATCH 18/31] HDDS-12742. Fix checkstyle Change-Id: I251e28caef8c20263cff822792cf2de8ca6b61ca --- .../main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 7a6312647ebe..11e6bc89bddc 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -622,10 +622,11 @@ TableIterator Date: Mon, 12 May 2025 20:03:37 -0400 Subject: [PATCH 19/31] HDDS-12742. Fix max buffer definition Change-Id: If5caeff1bcd4c5015b08f147a90be615dede3e38 --- .../utils/db/RDBStoreAbstractIterator.java | 50 +++++++++---- .../utils/db/RDBStoreCodecBufferIterator.java | 4 +- .../apache/hadoop/hdds/utils/db/RDBTable.java | 65 +++-------------- .../hadoop/hdds/utils/db/RawSpliterator.java | 13 ++-- .../hadoop/hdds/utils/db/TypedTable.java | 31 ++++---- .../db/TestRDBStoreByteArrayIterator.java | 62 +++++++++------- .../db/TestRDBStoreCodecBufferIterator.java | 71 ++++++++++--------- 7 files changed, 145 insertions(+), 151 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java index e3f4d8603c3b..ac2c28938791 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java @@ -32,7 +32,7 @@ * @param the raw type. */ abstract class RDBStoreAbstractIterator - implements TableIterator>> { + implements TableIterator> { private static final Logger LOG = LoggerFactory.getLogger(RDBStoreAbstractIterator.class); @@ -44,7 +44,7 @@ abstract class RDBStoreAbstractIterator // This is for schemas that use a fixed-length // prefix for each key. private final RAW prefix; - private boolean hasNext; + private Boolean hasNext; private boolean closed; RDBStoreAbstractIterator(ManagedRocksIterator iterator, RDBTable table, @@ -84,18 +84,22 @@ final RAW getPrefix() { @Override public final void forEachRemaining( - Consumer>> action) { + Consumer> action) { while (hasNext()) { - UncheckedAutoCloseableSupplier> entry = next(); + AutoCloseableRawKeyValue entry = next(); action.accept(entry); } } - private void setCurrentEntry() { + private void releaseEntry() { if (currentEntry != null) { currentEntry.release(); } + currentEntry = null; + hasNext = null; + } + private void setCurrentEntry() { boolean isValid = !closed && rocksDBIterator.get().isValid(); if (isValid) { currentEntry = getKeyValue(); @@ -112,16 +116,19 @@ public void setHasNext(boolean isValid, ReferenceCountedObject> @Override public final boolean hasNext() { + if (hasNext == null) { + setCurrentEntry(); + } return hasNext; } @Override - public final UncheckedAutoCloseableSupplier> next() { + public final AutoCloseableRawKeyValue next() { if (hasNext()) { - UncheckedAutoCloseableSupplier> entry = currentEntry.retainAndReleaseOnClose(); - this.previousKeyValue = entry.get(); + AutoCloseableRawKeyValue entry = new AutoCloseableRawKeyValue<>(currentEntry); + this.previousKeyValue = currentEntry.get(); rocksDBIterator.get().next(); - setCurrentEntry(); + releaseEntry(); return entry; } throw new NoSuchElementException("RocksDB Store has no more elements"); @@ -134,7 +141,7 @@ public final void seekToFirst() { } else { seek0(prefix); } - setCurrentEntry(); + releaseEntry(); } @Override @@ -144,16 +151,17 @@ public final void seekToLast() { } else { throw new UnsupportedOperationException("seekToLast: prefix != null"); } - setCurrentEntry(); + releaseEntry(); } @Override - public final UncheckedAutoCloseableSupplier> seek(RAW key) { + public final AutoCloseableRawKeyValue seek(RAW key) { seek0(key); + releaseEntry(); setCurrentEntry(); // Current entry should be only closed when the next() and thus closing the returned entry should be a noop. if (hasNext()) { - return currentEntry.retainAndReleaseOnClose(); + return new AutoCloseableRawKeyValue<>(currentEntry); } return null; } @@ -174,6 +182,20 @@ public final void removeFromDB() throws IOException { public void close() { rocksDBIterator.close(); closed = true; - setCurrentEntry(); + releaseEntry(); + } + + public static final class AutoCloseableRawKeyValue extends RawKeyValue implements AutoCloseable { + private final UncheckedAutoCloseableSupplier> keyValue; + + private AutoCloseableRawKeyValue(ReferenceCountedObject> kv) { + super(kv.get().getKey(), kv.get().getValue()); + this.keyValue = kv.retainAndReleaseOnClose(); + } + + @Override + public void close() { + keyValue.close(); + } } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java index 4ce61d43a91f..4621c18804a1 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java @@ -50,8 +50,8 @@ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator RDBStoreCodecBufferIterator(ManagedRocksIterator iterator, RDBTable table, 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); + // We need atleast 1 buffers one for setting next value and one for sending the current value. + maxNumberOfBuffersInMemory = Math.max(1, maxNumberOfBuffersInMemory); final String name = table != null ? table.getName() : null; this.availableBufferStack = new LinkedBlockingDeque<>(maxNumberOfBuffersInMemory); this.inUseBuffers = Sets.newConcurrentHashSet(); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index c9f7f6dce42d..0d5784896580 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -26,9 +26,9 @@ import java.util.function.Supplier; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; +import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily; import org.apache.hadoop.util.Time; -import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,7 +95,7 @@ public void putWithBatch(BatchOperation batch, byte[] key, byte[] value) @Override public boolean isEmpty() throws IOException { - try (TableIterator> keyIter = iterator()) { + try (TableIterator> keyIter = iterator()) { keyIter.seekToFirst(); return !keyIter.hasNext(); } @@ -211,68 +211,23 @@ public void deleteWithBatch(BatchOperation batch, byte[] key) } @Override - public TableIterator> iterator() + public TableIterator> iterator() throws IOException { return iterator((byte[])null); } @Override - public TableIterator> iterator(byte[] prefix) + public TableIterator> iterator(byte[] prefix) throws IOException { - RDBStoreByteArrayIterator itr = new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); - return new TableIterator>() { - @Override - public void seekToFirst() { - itr.seekToFirst(); - } - - @Override - public void seekToLast() { - itr.seekToLast(); - } - - @Override - public KeyValue seek(byte[] bytes) { - try (UncheckedAutoCloseableSupplier> kv = itr.seek(bytes)) { - return kv == null ? null : kv.get(); - } - } - - @Override - public void removeFromDB() throws IOException { - itr.removeFromDB(); - } - - @Override - public void close() { - itr.close(); - } - - @Override - public boolean hasNext() { - return itr.hasNext(); - } - - @Override - public KeyValue next() { - try (UncheckedAutoCloseableSupplier> kv = itr.next()) { - return kv.get(); - } - } - }; - } - - public TableIterator>> closeableSupplierIterator( - byte[] prefix) throws IOException { return new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); } - TableIterator>> iterator( + TableIterator> iterator( CodecBuffer prefix) throws IOException { - return iterator(prefix, 2); + return iterator(prefix, 1); } - TableIterator>> iterator( + TableIterator> iterator( CodecBuffer prefix, int maxNumberOfBuffers) throws IOException { return new RDBStoreCodecBufferIterator(db.newIterator(family, false), this, prefix, maxNumberOfBuffers); @@ -312,7 +267,7 @@ public List> getSequentialRangeKVs(byte[] startKey, @Override public void deleteBatchWithPrefix(BatchOperation batch, byte[] prefix) throws IOException { - try (TableIterator> iter + try (TableIterator> iter = iterator(prefix)) { while (iter.hasNext()) { deleteWithBatch(batch, iter.next().getKey()); @@ -323,7 +278,7 @@ public void deleteBatchWithPrefix(BatchOperation batch, byte[] prefix) @Override public void dumpToFileWithPrefix(File externalFile, byte[] prefix) throws IOException { - try (TableIterator> iter = iterator(prefix); + try (TableIterator> iter = iterator(prefix); RDBSstFileWriter fileWriter = new RDBSstFileWriter(externalFile)) { while (iter.hasNext()) { final KeyValue entry = iter.next(); @@ -348,7 +303,7 @@ private List> getRangeKVs(byte[] startKey, "Invalid count given " + count + ", count must be greater than 0"); } final List> result = new ArrayList<>(); - try (TableIterator> it + try (TableIterator> it = iterator(prefix)) { if (startKey == null) { it.seekToFirst(); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index f908c153f05b..c388cfc68534 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -24,8 +24,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.ratis.util.ReferenceCountedObject; -import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; /** * A {@link Table.KeyValueIterator} backed by a raw iterator. @@ -34,19 +34,18 @@ */ abstract class RawSpliterator implements Table.KeyValueSpliterator { - private final ReferenceCountedObject>>> rawIterator; + private final ReferenceCountedObject>> rawIterator; private final AtomicInteger maxNumberOfAdditionalSplits; private final Lock lock; private final AtomicReference closeException = new AtomicReference<>(); abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; - abstract TableIterator>> getRawIterator( + abstract TableIterator> getRawIterator( KEY prefix, KEY startKey, int maxParallelism) throws IOException; RawSpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { - TableIterator>> itr = getRawIterator(prefix, + TableIterator> itr = getRawIterator(prefix, startKey, maxParallelism); this.lock = new ReentrantLock(); this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); @@ -70,7 +69,7 @@ abstract TableIterator>> ge @Override public boolean tryAdvance(Consumer> action) { lock.lock(); - UncheckedAutoCloseableSupplier> kv = null; + AutoCloseableRawKeyValue kv = null; try { if (this.rawIterator.get().hasNext()) { kv = rawIterator.get().next(); @@ -80,7 +79,7 @@ public boolean tryAdvance(Consumer> action) { } try { if (kv != null) { - action.accept(convert(kv.get())); + action.accept(convert(kv)); return true; } } catch (IOException e) { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 11e6bc89bddc..9261a6d5a569 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.TableCacheMetrics; +import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheResult; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -43,7 +44,6 @@ import org.apache.hadoop.hdds.utils.db.cache.TableNoCache; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.function.CheckedBiFunction; -import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; /** * Strongly typed table implementation. @@ -427,7 +427,7 @@ public Table.KeyValueIterator iterator(KEY prefix) } } else { final byte[] prefixBytes = encodeKey(prefix); - return new TypedTableIterator(rawTable.closeableSupplierIterator(prefixBytes)); + return new TypedTableIterator(rawTable.iterator(prefixBytes)); } } @@ -568,7 +568,7 @@ public VALUE getValue() throws IOException { } private RawIterator newCodecBufferTableIterator( - TableIterator>> i) { + TableIterator> i) { return new RawIterator(i) { @Override AutoCloseSupplier convert(KEY key) throws IOException { @@ -610,14 +610,14 @@ KeyValue convert(RawKeyValue kv) throws IOException { } @Override - TableIterator>> getRawIterator( + TableIterator> getRawIterator( KEY prefix, KEY startKey, int maxParallelism) throws IOException { CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); try { - TableIterator>> itr = - rawTable.iterator(prefixBuffer, maxParallelism + 1); + TableIterator> itr = + rawTable.iterator(prefixBuffer, maxParallelism); if (startKeyBuffer != null) { itr.seek(startKeyBuffer); } @@ -649,12 +649,11 @@ KeyValue convert(RawKeyValue kv) throws IOException { } @Override - TableIterator>> getRawIterator( + TableIterator> getRawIterator( KEY prefix, KEY startKey, int maxParallelism) throws IOException { byte[] prefixBytes = encodeKey(prefix); byte[] startKeyBytes = encodeKey(startKey); - TableIterator>> itr = - rawTable.closeableSupplierIterator(prefixBytes); + TableIterator> itr = rawTable.iterator(prefixBytes); if (startKeyBytes != null) { itr.seek(startKeyBytes); } @@ -668,7 +667,7 @@ TableIterator>> getRa */ public class TypedTableIterator extends RawIterator { TypedTableIterator( - TableIterator>> rawIterator) { + TableIterator> rawIterator) { super(rawIterator); } @@ -691,9 +690,9 @@ KeyValue convert(RawKeyValue raw) { */ abstract class RawIterator implements Table.KeyValueIterator { - private final TableIterator>> rawIterator; + private final TableIterator> rawIterator; - RawIterator(TableIterator>> rawIterator) { + RawIterator(TableIterator> rawIterator) { this.rawIterator = rawIterator; } @@ -720,8 +719,8 @@ public void seekToLast() { @Override public KeyValue seek(KEY key) throws IOException { try (AutoCloseSupplier rawKey = convert(key); - UncheckedAutoCloseableSupplier> result = rawIterator.seek(rawKey.get())) { - return result == null ? null : convert(result.get()); + AutoCloseableRawKeyValue result = rawIterator.seek(rawKey.get())) { + return result == null ? null : convert(result); } } @@ -737,8 +736,8 @@ public boolean hasNext() { @Override public KeyValue next() { - try (UncheckedAutoCloseableSupplier> kv = rawIterator.next()) { - return convert(kv.get()); + try (AutoCloseableRawKeyValue kv = rawIterator.next()) { + return convert(kv); } catch (IOException e) { throw new IllegalStateException("Failed next()", e); } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java index d319d15fb3ed..2460dea79718 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreByteArrayIterator.java @@ -37,11 +37,11 @@ import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; import java.util.function.Consumer; +import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksObjectUtils; import org.apache.log4j.Level; import org.apache.log4j.Logger; -import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -90,33 +90,33 @@ public void testForeachRemainingCallsConsumerWithAllElements() throws IOExceptio .thenReturn(new byte[]{0x7f}, new byte[]{0x7e}, new byte[]{0x7d}) .thenThrow(new NoSuchElementException()); - final Consumer>> consumerStub + final Consumer> consumerStub = mock(Consumer.class); try (RDBStoreByteArrayIterator iter = newIterator()) { iter.forEachRemaining(consumerStub); - ArgumentCaptor>> capture = - forClass(UncheckedAutoCloseableSupplier.class); + ArgumentCaptor> capture = forClass(AutoCloseableRawKeyValue.class); verify(consumerStub, times(3)).accept(capture.capture()); assertArrayEquals( - new byte[]{0x00}, capture.getAllValues().get(0).get().getKey()); + new byte[]{0x00}, capture.getAllValues().get(0).getKey()); assertArrayEquals( - new byte[]{0x7f}, capture.getAllValues().get(0).get().getValue()); + new byte[]{0x7f}, capture.getAllValues().get(0).getValue()); assertArrayEquals( - new byte[]{0x01}, capture.getAllValues().get(1).get().getKey()); + new byte[]{0x01}, capture.getAllValues().get(1).getKey()); assertArrayEquals( - new byte[]{0x7e}, capture.getAllValues().get(1).get().getValue()); + new byte[]{0x7e}, capture.getAllValues().get(1).getValue()); assertArrayEquals( - new byte[]{0x02}, capture.getAllValues().get(2).get().getKey()); + new byte[]{0x02}, capture.getAllValues().get(2).getKey()); assertArrayEquals( - new byte[]{0x7d}, capture.getAllValues().get(2).get().getValue()); + new byte[]{0x7d}, capture.getAllValues().get(2).getValue()); } } @Test public void testHasNextDoesNotDependsOnIsvalid() { when(rocksDBIteratorMock.isValid()).thenReturn(true, false); - + when(rocksDBIteratorMock.key()).thenReturn(new byte[]{0x00}); + when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x00f}); try (RDBStoreByteArrayIterator iter = newIterator()) { assertTrue(iter.hasNext()); assertTrue(iter.hasNext()); @@ -129,7 +129,10 @@ public void testHasNextDoesNotDependsOnIsvalid() { @Test public void testNextCallsIsValidThenGetsTheValueAndStepsToNext() { + byte[] testKey = new byte[]{0x00}; when(rocksDBIteratorMock.isValid()).thenReturn(true); + when(rocksDBIteratorMock.key()).thenReturn(testKey); + when(rocksDBIteratorMock.value()).thenReturn(testKey); RDBStoreByteArrayIterator iter = newIterator(); InOrder verifier = inOrder(rocksDBIteratorMock); @@ -172,7 +175,7 @@ public void testSeekWithInvalidValue() { when(rocksDBIteratorMock.isValid()).thenReturn(false); try (RDBStoreByteArrayIterator iter = newIterator()) { - final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[] {0x55}); + final AutoCloseableRawKeyValue val = iter.seek(new byte[] {0x55}); assertFalse(iter.hasNext()); InOrder verifier = inOrder(rocksDBIteratorMock); verify(rocksDBIteratorMock, times(1)).seekToFirst(); //at construct time @@ -192,7 +195,7 @@ public void testSeekReturnsTheActualKey() throws Exception { when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x7f}); RDBStoreByteArrayIterator iter = newIterator(); - final UncheckedAutoCloseableSupplier> val = iter.seek(new byte[]{0x55}); + final AutoCloseableRawKeyValue val = iter.seek(new byte[]{0x55}); assertTrue(iter.hasNext()); InOrder verifier = inOrder(rocksDBIteratorMock); @@ -202,20 +205,20 @@ public void testSeekReturnsTheActualKey() throws Exception { verifier.verify(rocksDBIteratorMock, times(1)).isValid(); verifier.verify(rocksDBIteratorMock, times(1)).key(); verifier.verify(rocksDBIteratorMock, times(1)).value(); - assertArrayEquals(new byte[]{0x00}, val.get().getKey()); - assertArrayEquals(new byte[]{0x7f}, val.get().getValue()); + assertArrayEquals(new byte[]{0x00}, val.getKey()); + assertArrayEquals(new byte[]{0x7f}, val.getValue()); } @Test public void testGettingTheKeyIfIteratorIsValid() throws Exception { when(rocksDBIteratorMock.isValid()).thenReturn(true); when(rocksDBIteratorMock.key()).thenReturn(new byte[]{0x00}); - + when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x00}); RDBStoreByteArrayIterator iter = newIterator(); byte[] key = null; if (iter.hasNext()) { - try (UncheckedAutoCloseableSupplier> entry = iter.next()) { - key = entry.get().getKey(); + try (AutoCloseableRawKeyValue entry = iter.next()) { + key = entry.getKey(); } } @@ -233,13 +236,13 @@ public void testGettingTheValueIfIteratorIsValid() throws Exception { when(rocksDBIteratorMock.value()).thenReturn(new byte[]{0x7f}); try (RDBStoreByteArrayIterator iter = newIterator()) { - UncheckedAutoCloseableSupplier> entry; + AutoCloseableRawKeyValue entry; byte[] key = null; byte[] value = null; if (iter.hasNext()) { entry = iter.next(); - key = entry.get().getKey(); - value = entry.get().getValue(); + key = entry.getKey(); + value = entry.getValue(); } InOrder verifier = inOrder(rocksDBIteratorMock); @@ -255,6 +258,7 @@ public void testRemovingFromDBActuallyDeletesFromTable() throws Exception { byte[] testKey = new byte[]{0x00}; when(rocksDBIteratorMock.isValid()).thenReturn(true); when(rocksDBIteratorMock.key()).thenReturn(testKey); + when(rocksDBIteratorMock.value()).thenReturn(testKey); RDBStoreByteArrayIterator iter = newIterator(null); iter.next(); @@ -288,10 +292,15 @@ public void testNullPrefixedIterator() throws IOException { clearInvocations(rocksDBIteratorMock); when(rocksDBIteratorMock.isValid()).thenReturn(true); iter.seekToFirst(); - verify(rocksDBIteratorMock, times(1)).isValid(); - verify(rocksDBIteratorMock, times(1)).key(); + verify(rocksDBIteratorMock, times(0)).isValid(); + verify(rocksDBIteratorMock, times(0)).key(); verify(rocksDBIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksDBIteratorMock); + verify(rocksDBIteratorMock, times(0)).isValid(); + verify(rocksDBIteratorMock, times(0)).key(); + verify(rocksDBIteratorMock, times(0)).seekToFirst(); + iter.hasNext(); + clearInvocations(rocksDBIteratorMock); assertTrue(iter.hasNext()); verify(rocksDBIteratorMock, times(0)).isValid(); verify(rocksDBIteratorMock, times(0)).key(); @@ -311,9 +320,14 @@ public void testNormalPrefixedIterator() throws IOException { when(rocksDBIteratorMock.isValid()).thenReturn(true); when(rocksDBIteratorMock.key()).thenReturn(testPrefix); iter.seekToFirst(); + verify(rocksDBIteratorMock, times(0)).isValid(); + verify(rocksDBIteratorMock, times(0)).key(); + verify(rocksDBIteratorMock, times(1)).seek(testPrefix); + clearInvocations(rocksDBIteratorMock); + iter.hasNext(); verify(rocksDBIteratorMock, times(1)).isValid(); verify(rocksDBIteratorMock, times(1)).key(); - verify(rocksDBIteratorMock, times(1)).seek(testPrefix); + verify(rocksDBIteratorMock, times(0)).seek(testPrefix); clearInvocations(rocksDBIteratorMock); assertTrue(iter.hasNext()); // hasNext shouldn't make isValid() redundant calls. diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java index 489052240039..ec6d2820c7eb 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStoreCodecBufferIterator.java @@ -33,7 +33,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -47,12 +46,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdds.StringUtils; +import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksObjectUtils; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.apache.ozone.test.GenericTestUtils; -import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -88,8 +87,8 @@ RDBStoreCodecBufferIterator newIterator() { return newIterator(1); } - RDBStoreCodecBufferIterator newIterator(int numberOfThreads) { - return new RDBStoreCodecBufferIterator(managedRocksIterator, null, null, numberOfThreads); + RDBStoreCodecBufferIterator newIterator(int maxNumberOfBuffers) { + return new RDBStoreCodecBufferIterator(managedRocksIterator, null, null, maxNumberOfBuffers); } RDBStoreCodecBufferIterator newIterator(CodecBuffer prefix) { @@ -121,7 +120,7 @@ private int writeToBuffer(ByteBuffer buffer, byte... bytesToWrite) { @ValueSource(ints = {0, 1, 2, 3, 5, 10}) public void testRDBStoreCodecBufferIterGetsFailBeyondMaxBuffers(int maxBuffers) throws InterruptedException, TimeoutException { - List>> vals = new ArrayList<>(); + List> vals = new ArrayList<>(); when(rocksIteratorMock.isValid()).thenReturn(true); AtomicInteger counter = new AtomicInteger(0); @@ -130,45 +129,45 @@ public void testRDBStoreCodecBufferIterGetsFailBeyondMaxBuffers(int maxBuffers) when(rocksIteratorMock.value(any())) .thenAnswer(i -> writeToBuffer(i.getArgument(0), (byte)counter.getAndIncrement())); try (RDBStoreCodecBufferIterator iterator = newIterator(maxBuffers)) { - for (int i = 0; i < maxBuffers - 1; i++) { + for (int i = 0; i < maxBuffers; i++) { vals.add(iterator.next()); } - assertEquals(Math.max(maxBuffers - 1, 0), vals.size()); + assertEquals(Math.max(maxBuffers, 0), vals.size()); ExecutorService executor = Executors.newSingleThreadExecutor(); AtomicReference> nextThread = new AtomicReference<>(CompletableFuture.supplyAsync( () -> { - UncheckedAutoCloseableSupplier> v = iterator.next(); + AutoCloseableRawKeyValue v = iterator.next(); vals.add(v); return true; }, executor)); - if (maxBuffers <= 1) { - // Number of max buffers is always going to be 2. We need atleast 2 buffers one for getting the next value - // and one for returning the current value. + if (maxBuffers < 1) { + // Number of max buffers is always going to be at least 1. We need at least 1 buffers one for getting the next + // value and one for returning the current value. GenericTestUtils.waitFor(() -> nextThread.get().isDone() && nextThread.get().join(), 10, 100); System.out.println(Thread.currentThread().getName()); nextThread.set(CompletableFuture.supplyAsync( () -> { - UncheckedAutoCloseableSupplier> v = iterator.next(); + AutoCloseableRawKeyValue v = iterator.next(); vals.add(v); return true; }, executor)); } - assertEquals(Math.max(maxBuffers - 1, 1), vals.size()); + assertEquals(Math.max(1, maxBuffers), vals.size()); for (int i = 0; i < vals.size(); i++) { - assertEquals(2 * i, vals.get(i).get().getKey().getArray()[0]); - assertEquals(2 * i + 1, vals.get(i).get().getValue().getArray()[0]); + assertEquals(2 * i, vals.get(i).getKey().getArray()[0]); + assertEquals(2 * i + 1, vals.get(i).getValue().getArray()[0]); } assertFalse(nextThread.get().isDone()); int size = vals.size(); vals.get(0).close(); GenericTestUtils.waitFor(() -> nextThread.get().isDone() && nextThread.get().join(), 10, 100); assertEquals(size + 1, vals.size()); - assertEquals(2 * size, vals.get(size).get().getKey().getArray()[0]); - assertEquals(2 * size + 1, vals.get(size).get().getValue().getArray()[0]); - for (UncheckedAutoCloseableSupplier> v : vals) { + assertEquals(2 * size, vals.get(size).getKey().getArray()[0]); + assertEquals(2 * size + 1, vals.get(size).getValue().getArray()[0]); + for (AutoCloseableRawKeyValue v : vals) { v.close(); } executor.shutdown(); @@ -194,11 +193,8 @@ public void testForEachRemaining() throws Exception { try (RDBStoreCodecBufferIterator i = newIterator()) { i.forEachRemaining(kvSupplier -> { try { - Table.KeyValue kv = kvSupplier.get(); remaining.add(RawKeyValue.create( - kv.getKey().getArray(), kv.getValue().getArray())); - } catch (IOException e) { - throw new RuntimeException(e); + kvSupplier.getKey().getArray(), kvSupplier.getValue().getArray())); } finally { kvSupplier.close(); } @@ -285,7 +281,7 @@ public void testSeekWithInvalidValue() { try (RDBStoreCodecBufferIterator i = newIterator(); CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); - UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { + AutoCloseableRawKeyValue valSupplier = i.seek(target)) { assertFalse(i.hasNext()); InOrder verifier = inOrder(rocksIteratorMock); verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time @@ -309,7 +305,7 @@ public void testSeekReturnsTheActualKey() throws Exception { try (RDBStoreCodecBufferIterator i = newIterator(); CodecBuffer target = CodecBuffer.wrap(new byte[] {0x55}); - UncheckedAutoCloseableSupplier> valSupplier = i.seek(target)) { + AutoCloseableRawKeyValue valSupplier = i.seek(target)) { assertTrue(i.hasNext()); InOrder verifier = inOrder(rocksIteratorMock); verify(rocksIteratorMock, times(1)).seekToFirst(); //at construct time @@ -319,8 +315,8 @@ public void testSeekReturnsTheActualKey() throws Exception { verifier.verify(rocksIteratorMock, times(1)).isValid(); verifier.verify(rocksIteratorMock, times(1)).key(any()); verifier.verify(rocksIteratorMock, times(1)).value(any()); - assertArrayEquals(new byte[] {0x00}, valSupplier.get().getKey().getArray()); - assertArrayEquals(new byte[] {0x7f}, valSupplier.get().getValue().getArray()); + assertArrayEquals(new byte[] {0x00}, valSupplier.getKey().getArray()); + assertArrayEquals(new byte[] {0x7f}, valSupplier.getValue().getArray()); } CodecTestUtil.gc(); @@ -335,8 +331,8 @@ public void testGettingTheKeyIfIteratorIsValid() throws Exception { byte[] key = null; try (RDBStoreCodecBufferIterator i = newIterator()) { if (i.hasNext()) { - try (UncheckedAutoCloseableSupplier> kv = i.next()) { - key = kv.get().getKey().getArray(); + try (AutoCloseableRawKeyValue kv = i.next()) { + key = kv.getKey().getArray(); } } } @@ -362,9 +358,9 @@ public void testGettingTheValueIfIteratorIsValid() throws Exception { byte[] value = null; try (RDBStoreCodecBufferIterator i = newIterator()) { if (i.hasNext()) { - try (UncheckedAutoCloseableSupplier> entry = i.next()) { - key = entry.get().getKey().getArray(); - value = entry.get().getValue().getArray(); + try (AutoCloseableRawKeyValue entry = i.next()) { + key = entry.getKey().getArray(); + value = entry.getValue().getArray(); } } } @@ -427,9 +423,13 @@ public void testNullPrefixedIterator() throws Exception { clearInvocations(rocksIteratorMock); when(rocksIteratorMock.isValid()).thenReturn(true); i.seekToFirst(); + verify(rocksIteratorMock, times(0)).isValid(); + verify(rocksIteratorMock, times(0)).key(any()); + verify(rocksIteratorMock, times(1)).seekToFirst(); + clearInvocations(rocksIteratorMock); + i.hasNext(); verify(rocksIteratorMock, times(1)).isValid(); verify(rocksIteratorMock, times(1)).key(any()); - verify(rocksIteratorMock, times(1)).seekToFirst(); clearInvocations(rocksIteratorMock); assertTrue(i.hasNext()); @@ -456,9 +456,14 @@ public void testNormalPrefixedIterator() throws Exception { when(rocksIteratorMock.key(any())) .then(newAnswer("key1", prefixBytes)); i.seekToFirst(); + verify(rocksIteratorMock, times(0)).isValid(); + verify(rocksIteratorMock, times(0)).key(any()); + verify(rocksIteratorMock, times(1)).seek(prefix); + clearInvocations(rocksIteratorMock); + i.hasNext(); verify(rocksIteratorMock, times(1)).isValid(); verify(rocksIteratorMock, times(1)).key(any()); - verify(rocksIteratorMock, times(1)).seek(prefix); + verify(rocksIteratorMock, times(0)).seek(prefix); clearInvocations(rocksIteratorMock); assertTrue(i.hasNext()); From 86fe1018d3b846144793af8ab9c2bc512e248316 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 18 May 2025 17:22:12 -0400 Subject: [PATCH 20/31] HDDS-12742. Add tests Change-Id: Iaf7e8059d442e954e722c58620ccd4811e15c855 --- .../hadoop/hdds/utils/db/RawSpliterator.java | 18 +- .../apache/hadoop/hdds/utils/db/Table.java | 17 +- .../hdds/utils/db/TableSpliterator.java | 25 --- .../hdds/utils/db/RawSpliteratorTest.java | 179 ++++++++++++++++++ 4 files changed, 209 insertions(+), 30 deletions(-) delete mode 100644 hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java create mode 100644 hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index c388cfc68534..869f2a75939b 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -28,9 +28,19 @@ import org.apache.ratis.util.ReferenceCountedObject; /** - * A {@link Table.KeyValueIterator} backed by a raw iterator. + * An abstract implementation of {@link Table.KeyValueSpliterator} designed + * for iterating and splitting over raw key-value pairs retrieved via a + * {@link TableIterator}. * - * @param The raw type. + *

      This class manages asynchronous raw iterator resources and provides + * functionality for converting raw key-value pairs into a structured + * {@link Table.KeyValue} representation. It also allows controlled splitting + * of the underlying iterator for parallel processing, while ensuring proper + * resource handling and thread safety. + * + * @param The raw representation type of the key-value pair. + * @param The type of key in the key-value pair. + * @param The type of value in the key-value pair. */ abstract class RawSpliterator implements Table.KeyValueSpliterator { @@ -100,7 +110,7 @@ public int characteristics() { @Override public Spliterator> trySplit() { int val = maxNumberOfAdditionalSplits.decrementAndGet(); - if (val >= 1) { + if (val >= 0) { try { this.rawIterator.retain(); } catch (Exception e) { @@ -108,6 +118,8 @@ public Spliterator> trySplit() { return null; } return this; + } else { + maxNumberOfAdditionalSplits.incrementAndGet(); } return null; } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java index 455c90661c38..9d64cd57b219 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java @@ -17,12 +17,14 @@ package org.apache.hadoop.hdds.utils.db; +import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Spliterator; import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; @@ -421,7 +423,18 @@ interface KeyValueIterator extends TableIterator> { } - /** A {@link org.apache.hadoop.hdds.utils.db.TableSpliterator} to split iterate {@link KeyValue}s. */ - interface KeyValueSpliterator extends TableSpliterator> { + /** + * A generic interface extending {@link Spliterator} and {@link Closeable}, + * designed for iterating over and splitting key-value pairs. + * + * @param The type of keys in the key-value pairs. + * @param The type of values in the key-value pairs. + * + * This interface facilitates traversal and splitting of key-value pairs + * while also providing resource management capabilities via {@link Closeable}. + * Implementations must handle key-value pair splitting to enable parallel processing, + * and ensure proper resource management when the spliterator is closed. + */ + interface KeyValueSpliterator extends Spliterator>, Closeable { } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java deleted file mode 100644 index 3f435b2ff130..000000000000 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableSpliterator.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdds.utils.db; - -import java.io.Closeable; -import java.util.Spliterator; - -interface TableSpliterator extends Spliterator, Closeable { - -} diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java new file mode 100644 index 000000000000..92ff8f940dd7 --- /dev/null +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.utils.db; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Spliterator; +import java.util.function.Consumer; +import org.junit.jupiter.api.Test; + +/** + * Unit test class for verifying the behavior of the {@code RawSpliterator}. + * This class contains tests to evaluate the functionality of its methods, + * such as {@code tryAdvance}, {@code trySplit}, and error handling during + * method execution. + */ +class RawSpliteratorTest { + + @Test + void testTryAdvanceWithValidElement() throws IOException { + // Mock dependencies + TableIterator> rawIteratorMock = + mock(TableIterator.class); + RDBStoreAbstractIterator.AutoCloseableRawKeyValue rawKeyValueMock = + mock(RDBStoreAbstractIterator.AutoCloseableRawKeyValue.class); + + when(rawIteratorMock.hasNext()).thenReturn(true); + when(rawIteratorMock.next()).thenReturn(rawKeyValueMock); + + RawSpliterator rawSpliterator = new MockRawSpliterator(1) { + @Override + TableIterator> getRawIterator(String prefix, + String startKey, int maxParallelism) { + return rawIteratorMock; + } + }; + + when(rawKeyValueMock.getKey()).thenReturn("key"); + when(rawKeyValueMock.getValue()).thenReturn("value"); + + Consumer> action = keyValue -> { + try { + assertEquals("key", keyValue.getKey()); + assertEquals("value", keyValue.getValue()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + + boolean result = rawSpliterator.tryAdvance(action); + + assertTrue(result); + verify(rawIteratorMock).hasNext(); + verify(rawIteratorMock).next(); + verify(rawKeyValueMock).close(); + } + + @Test + void testTryAdvanceWithNoElement() throws IOException { + TableIterator> rawIteratorMock = + mock(TableIterator.class); + + when(rawIteratorMock.hasNext()).thenReturn(false); + + RawSpliterator rawSpliterator = new MockRawSpliterator(1) { + @Override + TableIterator> getRawIterator(String prefix, + String startKey, int maxParallelism) { + return rawIteratorMock; + } + }; + + Consumer> action = keyValue -> fail("Action should not be called"); + + boolean result = rawSpliterator.tryAdvance(action); + + assertFalse(result); + verify(rawIteratorMock).hasNext(); + verify(rawIteratorMock, never()).next(); + } + + @Test + void testTryAdvanceWhenConvertThrowsIOException() throws IOException { + TableIterator> rawIteratorMock = + mock(TableIterator.class); + RDBStoreAbstractIterator.AutoCloseableRawKeyValue rawKeyValueMock = + mock(RDBStoreAbstractIterator.AutoCloseableRawKeyValue.class); + + when(rawIteratorMock.hasNext()).thenReturn(true); + when(rawIteratorMock.next()).thenReturn(rawKeyValueMock); + + RawSpliterator rawSpliterator = new RawSpliterator(null, null, 1) { + + @Override + Table.KeyValue convert(RawKeyValue kv) throws IOException { + throw new IOException("Mocked exception"); + } + + @Override + TableIterator> getRawIterator(String prefix, + String startKey, int maxParallelism) { + return rawIteratorMock; + } + }; + + Consumer> action = keyValue -> { + }; + + Exception exception = assertThrows(IllegalStateException.class, () -> rawSpliterator.tryAdvance(action)); + + assertEquals("Failed next()", exception.getMessage()); + assertNotNull(exception.getCause()); + assertEquals(IOException.class, exception.getCause().getClass()); + assertEquals("Mocked exception", exception.getCause().getMessage()); + verify(rawIteratorMock).hasNext(); + verify(rawIteratorMock).next(); + } + + @Test + void testTrySplits() throws IOException { + TableIterator> rawIteratorMock = + mock(TableIterator.class); + + RawSpliterator rawSpliterator = new MockRawSpliterator(2) { + @Override + TableIterator> getRawIterator(String prefix, + String startKey, int maxParallelism) { + return rawIteratorMock; + } + }; + Spliterator> split1 = rawSpliterator.trySplit(); + Spliterator> split2 = rawSpliterator.trySplit(); + assertNotNull(split1); + assertNull(split2); + rawSpliterator.close(); + Spliterator> split3 = split1.trySplit(); + assertNotNull(split3); + } + + private abstract static class MockRawSpliterator extends RawSpliterator { + + MockRawSpliterator(int maxParallelism) throws IOException { + super(null, null, maxParallelism); + + } + + @Override + Table.KeyValue convert(RawKeyValue kv) { + return Table.newKeyValue(kv.getKey(), kv.getValue()); + } + } +} From f8257545ac94878b3ffb79f50d54a6134f3ed1d7 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 21 May 2025 06:49:30 -0400 Subject: [PATCH 21/31] HDDS-12742. Fix exception handling for memory leaks Change-Id: Id6f11892e47103ebecd7b2cd8549d4ff4ebcf444 --- .../hadoop/hdds/utils/db/RawSpliterator.java | 77 ++++++++++++------- .../hadoop/hdds/utils/db/TypedTable.java | 22 +++--- .../hdds/utils/db/RawSpliteratorTest.java | 5 +- 3 files changed, 64 insertions(+), 40 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index 869f2a75939b..b064ee35e2d5 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -48,58 +48,61 @@ abstract class RawSpliterator implements Table.KeyValueSplitera private final AtomicInteger maxNumberOfAdditionalSplits; private final Lock lock; private final AtomicReference closeException = new AtomicReference<>(); + private boolean closed; + private final boolean closeOnException; abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; abstract TableIterator> getRawIterator( KEY prefix, KEY startKey, int maxParallelism) throws IOException; - RawSpliterator(KEY prefix, KEY startKey, int maxParallelism) throws IOException { + RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { TableIterator> itr = getRawIterator(prefix, startKey, maxParallelism); - this.lock = new ReentrantLock(); - this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); - this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, - (completelyReleased) -> { - if (completelyReleased) { - lock.lock(); - try { - itr.close(); - } catch (IOException e) { - closeException.set(e); - } finally { - lock.unlock(); + try { + this.closeOnException = closeOnException; + this.lock = new ReentrantLock(); + this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); + this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, + (completelyReleased) -> { + if (completelyReleased) { + closeRawIterator(true); } - } - this.maxNumberOfAdditionalSplits.incrementAndGet(); - }); - this.rawIterator.retain(); + this.maxNumberOfAdditionalSplits.incrementAndGet(); + }); + this.rawIterator.retain(); + } catch (Throwable e) { + itr.close(); + throw e; + } } @Override public boolean tryAdvance(Consumer> action) { lock.lock(); - AutoCloseableRawKeyValue kv = null; + AutoCloseableRawKeyValue kv; try { - if (this.rawIterator.get().hasNext()) { + if (!closed && this.rawIterator.get().hasNext()) { kv = rawIterator.get().next(); + } else { + closeRawIterator(false); + return false; } } finally { lock.unlock(); } - try { - if (kv != null) { - action.accept(convert(kv)); + try (AutoCloseableRawKeyValue keyValue = kv) { + if (keyValue != null) { + action.accept(convert(keyValue)); return true; } - } catch (IOException e) { - throw new IllegalStateException("Failed next()", e); - } finally { - if (kv != null) { - kv.close(); + return false; + } catch (Throwable e) { + if (closeOnException) { + closeRawIterator(true); } + throw new IllegalStateException("Failed next()", e); } - return false; } @Override @@ -124,6 +127,24 @@ public Spliterator> trySplit() { return null; } + private void closeRawIterator(boolean acquireLock) { + if (!closed) { + if (acquireLock) { + this.lock.lock(); + } + try { + closed = true; + this.rawIterator.get().close(); + } catch (IOException e) { + closeException.set(e); + } finally { + if (acquireLock) { + this.lock.unlock(); + } + } + } + } + @Override public void close() throws IOException { this.rawIterator.release(); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 9261a6d5a569..7a610d2b4674 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -400,16 +400,18 @@ public Table.KeyValueIterator iterator() throws IOException { return iterator(null); } - public Table.KeyValueSpliterator spliterator(int maxParallelism) throws IOException { - return spliterator(null, null, maxParallelism); + public Table.KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) + throws IOException { + return spliterator(null, null, maxParallelism, closeOnException); } - public Table.KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism) + public Table.KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism, + boolean closeOnException) throws IOException { if (supportCodecBuffer) { - return newCodecBufferSpliterator(prefix, startKey, maxParallelism); + return newCodecBufferSpliterator(prefix, startKey, maxParallelism, closeOnException); } - return newByteArraySpliterator(prefix, startKey, maxParallelism); + return newByteArraySpliterator(prefix, startKey, maxParallelism, closeOnException); } @Override @@ -598,8 +600,8 @@ KeyValue convert(RawKeyValue raw) } private RawSpliterator newCodecBufferSpliterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism) { + KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { @Override KeyValue convert(RawKeyValue kv) throws IOException { @@ -636,9 +638,9 @@ TableIterator> getRawIterator }; } - private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism) - throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism) { + private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism, + boolean closeOnException) throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { @Override KeyValue convert(RawKeyValue kv) throws IOException { diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java index 92ff8f940dd7..7b50cc8b0dc2 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java @@ -116,7 +116,8 @@ void testTryAdvanceWhenConvertThrowsIOException() throws IOException { when(rawIteratorMock.hasNext()).thenReturn(true); when(rawIteratorMock.next()).thenReturn(rawKeyValueMock); - RawSpliterator rawSpliterator = new RawSpliterator(null, null, 1) { + RawSpliterator rawSpliterator = new RawSpliterator(null, null, 1, + true) { @Override Table.KeyValue convert(RawKeyValue kv) throws IOException { @@ -167,7 +168,7 @@ TableIterator> private abstract static class MockRawSpliterator extends RawSpliterator { MockRawSpliterator(int maxParallelism) throws IOException { - super(null, null, maxParallelism); + super(null, null, maxParallelism, true); } From e4155a6a293713c1b89e0f29b932359ca3bebb2d Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 23 May 2025 11:31:18 -0400 Subject: [PATCH 22/31] HDDS-12742. Make spliterator an interface parameter Change-Id: I214cab272e61ce1de527b78190dd1c5560061c8e --- .../apache/hadoop/hdds/utils/db/RDBTable.java | 34 +++++++++++++++++++ .../apache/hadoop/hdds/utils/db/Table.java | 28 +++++++++++++++ .../hadoop/hdds/utils/db/TypedTable.java | 5 +-- 3 files changed, 65 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index 0d5784896580..dc9a2857b416 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -222,6 +222,18 @@ public TableIterator> iterator(byte[] p return new RDBStoreByteArrayIterator(db.newIterator(family, false), this, prefix); } + @Override + public KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) + throws IOException { + return spliterator(null, null, maxParallelism, closeOnException); + } + + @Override + public KeyValueSpliterator spliterator(byte[] startKey, byte[] prefix, int maxParallelism, + boolean closeOnException) throws IOException { + return newByteArraySpliterator(prefix, startKey, maxParallelism, closeOnException); + } + TableIterator> iterator( CodecBuffer prefix) throws IOException { return iterator(prefix, 1); @@ -362,4 +374,26 @@ && get(startKey) == null) { } return result; } + + private RawSpliterator newByteArraySpliterator(byte[] prefix, byte[] startKey, + int maxParallelism, boolean closeOnException) throws IOException { + return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { + + @Override + KeyValue convert(RawKeyValue kv) { + final int rawSize = kv.getValue().length; + return Table.newKeyValue(kv.getKey(), kv.getValue(), rawSize); + } + + @Override + TableIterator> getRawIterator( + byte[] prefix, byte[] startKey, int maxParallelism) throws IOException { + TableIterator> itr = iterator(prefix); + if (startKey != null) { + itr.seek(startKey); + } + return itr; + } + }; + } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java index dae6da445331..8ca7258041b7 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java @@ -172,6 +172,34 @@ default VALUE getReadCopy(KEY key) throws IOException { TableIterator> iterator(KEY prefix) throws IOException; + /** + * Creates a spliterator over the key-value pairs in the table with the specified + * maximum parallelism and exception handling behavior. + * + * @param maxParallelism the maximum number of threads that can be used for parallel processing. + * @param closeOnException if true, the spliterator will automatically close resources + * if an exception occurs during processing. + * @return a {@code Table.KeyValueSpliterator} instance for iterating over the key-value pairs. + * @throws IOException if an I/O error occurs during spliterator creation. + */ + Table.KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) + throws IOException; + + /** + * Returns a spliterator that iterates over the key-value pairs starting from a given key + * and within a specified key prefix. The spliterator can be parallelized up to a given level + * of parallelism and may optionally close resources in case of exceptions. + * + * @param startKey the starting key from which iteration begins + * @param prefix the key prefix used to limit the keys being iterated + * @param maxParallelism the maximum level of parallelism allowed for the iteration + * @param closeOnException if true, closes resources when an exception occurs during iteration + * @return a spliterator that supports parallel iteration over the key-value pairs + * @throws IOException if an I/O error occurs during the creation of the spliterator + */ + Table.KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism, + boolean closeOnException) throws IOException; + /** * Returns the Name of this Table. * @return - Table Name. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 7a610d2b4674..bb150e6060dc 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -400,14 +400,15 @@ public Table.KeyValueIterator iterator() throws IOException { return iterator(null); } + @Override public Table.KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) throws IOException { return spliterator(null, null, maxParallelism, closeOnException); } + @Override public Table.KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism, - boolean closeOnException) - throws IOException { + boolean closeOnException) throws IOException { if (supportCodecBuffer) { return newCodecBufferSpliterator(prefix, startKey, maxParallelism, closeOnException); } From 985d612f99c868ca7519711c2d6df574c6d229af Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Mon, 26 May 2025 20:38:07 -0400 Subject: [PATCH 23/31] HDDS-12742. Fix build Change-Id: I8b80f584a316cc54778068572702fe3c69699185 --- .../ozone/container/metadata/DatanodeTable.java | 15 +++++++++++++++ .../hadoop/hdds/utils/db/InMemoryTestTable.java | 11 +++++++++++ 2 files changed, 26 insertions(+) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java index ec9359fb3725..a3c12349a899 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/metadata/DatanodeTable.java @@ -89,6 +89,21 @@ public void deleteWithBatch(BatchOperation batch, KEY key) "version."); } + @Override + public KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) { + throw new UnsupportedOperationException("Iterating tables directly is not" + + " supported for datanode containers due to differing schema " + + "version."); + } + + @Override + public KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism, + boolean closeOnException) { + throw new UnsupportedOperationException("Iterating tables directly is not" + + " supported for datanode containers due to differing schema " + + "version."); + } + @Override public String getName() { return table.getName(); diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/InMemoryTestTable.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/InMemoryTestTable.java index 62e15a08662b..24ae9d9de091 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/InMemoryTestTable.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/InMemoryTestTable.java @@ -99,6 +99,17 @@ public void deleteRange(KEY beginKey, KEY endKey) { throw new UnsupportedOperationException(); } + @Override + public KeyValueSpliterator spliterator(int maxParallelism, boolean closeOnException) { + throw new UnsupportedOperationException(); + } + + @Override + public KeyValueSpliterator spliterator(KEY startKey, KEY prefix, int maxParallelism, + boolean closeOnException) { + throw new UnsupportedOperationException(); + } + @Override public String getName() { return ""; From b98968b9065c0ccba4b1cab2ec9ad49cb3d28f34 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Mon, 26 May 2025 22:55:21 -0400 Subject: [PATCH 24/31] HDDS-12742. Fix findbugs Change-Id: I6527e2a2ed8f22131880e5c7cf14601614254671 --- .../hadoop/hdds/utils/db/RawSpliterator.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index b064ee35e2d5..626dce515751 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -66,7 +66,7 @@ abstract TableIterator> getRawIterator( this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, (completelyReleased) -> { if (completelyReleased) { - closeRawIterator(true); + closeRawIteratorWithLock(); } this.maxNumberOfAdditionalSplits.incrementAndGet(); }); @@ -85,7 +85,7 @@ public boolean tryAdvance(Consumer> action) { if (!closed && this.rawIterator.get().hasNext()) { kv = rawIterator.get().next(); } else { - closeRawIterator(false); + closeRawIterator(); return false; } } finally { @@ -99,7 +99,7 @@ public boolean tryAdvance(Consumer> action) { return false; } catch (Throwable e) { if (closeOnException) { - closeRawIterator(true); + closeRawIteratorWithLock(); } throw new IllegalStateException("Failed next()", e); } @@ -127,20 +127,24 @@ public Spliterator> trySplit() { return null; } - private void closeRawIterator(boolean acquireLock) { + private void closeRawIterator() { if (!closed) { - if (acquireLock) { - this.lock.lock(); - } try { closed = true; this.rawIterator.get().close(); } catch (IOException e) { closeException.set(e); + } + } + } + + private void closeRawIteratorWithLock() { + if (!closed) { + this.lock.lock(); + try { + closeRawIterator(); } finally { - if (acquireLock) { - this.lock.unlock(); - } + this.lock.unlock(); } } } From af6d30d53811aadf6d98ddc06a33cc8d7bfb41bd Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 03:32:01 -0400 Subject: [PATCH 25/31] HDDS-12779. Parallelize table Spliterator with multiple multiple iterators across different ranges Change-Id: I8ee22aed8c4be85ccb693cc9dcaf00880c4e736b --- .../hadoop/hdds/utils/db/CodecBuffer.java | 15 +- .../hadoop/hdds/utils/db/BaseRDBTable.java | 29 +++ .../hadoop/hdds/utils/db/ByteArrayCodec.java | 21 ++ .../apache/hadoop/hdds/utils/db/RDBTable.java | 86 ++++++-- .../hadoop/hdds/utils/db/RawSpliterator.java | 77 ++++++- .../hadoop/hdds/utils/db/TypedTable.java | 208 +++++++++++++----- .../hdds/utils/db/RawSpliteratorTest.java | 68 +++++- 7 files changed, 420 insertions(+), 84 deletions(-) create mode 100644 hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/BaseRDBTable.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/CodecBuffer.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/CodecBuffer.java index e9108112bd49..4d83e64d6ea1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/CodecBuffer.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/db/CodecBuffer.java @@ -50,7 +50,7 @@ * A buffer used by {@link Codec} * for supporting RocksDB direct {@link ByteBuffer} APIs. */ -public class CodecBuffer implements UncheckedAutoCloseable { +public class CodecBuffer implements UncheckedAutoCloseable, Comparable { private static final Logger LOG = LoggerFactory.getLogger(CodecBuffer.class); private static final ByteBufAllocator POOL = PooledByteBufAllocator.DEFAULT; @@ -376,6 +376,19 @@ public boolean startsWith(CodecBuffer prefix) { return buf.slice(buf.readerIndex(), length).equals(prefix.buf); } + public int compareTo(byte[] other) { + Objects.requireNonNull(other, "other == null"); + final int size = Math.min(readableBytes(), other.length); + for (int i = 0; i < size; i++) { + final int b1 = buf.getByte(buf.readerIndex() + i) & 0xff; + final int b2 = other[i] & 0xff; + if (b1 != b2) { + return b1 - b2; + } + } + return readableBytes() - other.length; + } + /** @return an {@link InputStream} reading from this buffer. */ public InputStream getInputStream() { return new ByteBufInputStream(buf.duplicate()); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/BaseRDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/BaseRDBTable.java new file mode 100644 index 000000000000..7ff912b3df66 --- /dev/null +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/BaseRDBTable.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.utils.db; + +import java.io.IOException; +import java.util.List; +import org.rocksdb.LiveFileMetaData; + +/** + * Base table interface for Rocksdb. + */ +public interface BaseRDBTable extends Table { + List getTableSstFiles() throws IOException; +} diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/ByteArrayCodec.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/ByteArrayCodec.java index 7ae65652d18f..c40a6aeea29e 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/ByteArrayCodec.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/ByteArrayCodec.java @@ -17,17 +17,24 @@ package org.apache.hadoop.hdds.utils.db; +import java.util.Comparator; + /** * No-op codec for byte arrays. */ public final class ByteArrayCodec implements Codec { private static final Codec INSTANCE = new ByteArrayCodec(); + private static final Comparator COMPARATOR = new ByteWiseComparator(); public static Codec get() { return INSTANCE; } + public static Comparator getComparator() { + return COMPARATOR; + } + private ByteArrayCodec() { // singleton } @@ -51,4 +58,18 @@ public byte[] fromPersistedFormat(byte[] bytes) { public byte[] copyObject(byte[] bytes) { return bytes; } + + private static class ByteWiseComparator implements Comparator { + @Override + public int compare(byte[] o1, byte[] o2) { + int length = Math.min(o1.length, o2.length); + for (int i = 0; i < length; i++) { + int compareValue = Integer.compareUnsigned(Byte.toUnsignedInt(o1[i]), Byte.toUnsignedInt(o2[i])); + if (compareValue != 0) { + return compareValue; + } + } + return Integer.compare(o1.length, o2.length); + } + } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index dc9a2857b416..26e52f643ce3 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -17,18 +17,24 @@ package org.apache.hadoop.hdds.utils.db; +import static org.apache.hadoop.hdds.utils.db.RocksDatabase.bytes2String; + import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Spliterator; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily; import org.apache.hadoop.util.Time; +import org.rocksdb.LiveFileMetaData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +44,7 @@ * metadata store content. All other user's using Table should use TypedTable. */ @InterfaceAudience.Private -class RDBTable implements Table { +class RDBTable implements BaseRDBTable { private static final Logger LOG = LoggerFactory.getLogger(RDBTable.class); @@ -377,23 +383,71 @@ && get(startKey) == null) { private RawSpliterator newByteArraySpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { + return new ByteArrayRawSpliterator(prefix, startKey, maxParallelism, closeOnException); + } - @Override - KeyValue convert(RawKeyValue kv) { - final int rawSize = kv.getValue().length; - return Table.newKeyValue(kv.getKey(), kv.getValue(), rawSize); - } + @Override + public List getTableSstFiles() throws IOException { + return this.db.getSstFileList().stream() + .filter(liveFileMetaData -> getName().equals(bytes2String(liveFileMetaData.columnFamilyName()))) + .collect(Collectors.toList()); + } - @Override - TableIterator> getRawIterator( - byte[] prefix, byte[] startKey, int maxParallelism) throws IOException { - TableIterator> itr = iterator(prefix); - if (startKey != null) { - itr.seek(startKey); - } - return itr; + private final class ByteArrayRawSpliterator extends RawSpliterator { + + private ByteArrayRawSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + } + + private ByteArrayRawSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException, + List boundKeys) throws IOException { + super(prefix, startKey, maxParallelism, closeOnException, boundKeys); + } + + @Override + KeyValue convert(RawKeyValue kv) { + final int rawSize = kv.getValue().length; + return Table.newKeyValue(kv.getKey(), kv.getValue(), rawSize); + } + + @Override + List getBoundaryKeys(byte[] prefix, byte[] startKey) throws IOException { + return getTableSstFiles().stream() + .flatMap(liveFileMetaData -> Stream.of(liveFileMetaData.smallestKey(), liveFileMetaData.largestKey())) + .filter(value -> { + if (value.length < prefix.length) { + return false; + } + for (int i = 0; i < prefix.length; i++) { + if (value[i] != prefix[i]) { + return false; + } + } + return true; + }).filter(value -> ByteArrayCodec.getComparator().compare(value, startKey) >= 0) + .collect(Collectors.toList()); + } + + @Override + int compare(byte[] value1, byte[] value2) { + return ByteArrayCodec.getComparator().compare(value1, value2); + } + + @Override + TableIterator> getRawIterator( + byte[] prefix, byte[] startKey, int maxParallelism) throws IOException { + TableIterator> itr = iterator(prefix); + if (startKey != null) { + itr.seek(startKey); } - }; + return itr; + } + + @Override + Spliterator> createNewSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, + boolean closeOnException, List boundaryKeys) throws IOException { + return new ByteArrayRawSpliterator(prefix, startKey, maxParallelism, closeOnException, boundaryKeys); + } } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index 626dce515751..dece75293baf 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -18,12 +18,14 @@ package org.apache.hadoop.hdds.utils.db; import java.io.IOException; +import java.util.List; import java.util.Spliterator; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import java.util.stream.Collectors; import org.apache.hadoop.hdds.utils.db.RDBStoreAbstractIterator.AutoCloseableRawKeyValue; import org.apache.ratis.util.ReferenceCountedObject; @@ -44,23 +46,57 @@ */ abstract class RawSpliterator implements Table.KeyValueSpliterator { - private final ReferenceCountedObject>> rawIterator; - private final AtomicInteger maxNumberOfAdditionalSplits; - private final Lock lock; + private ReferenceCountedObject>> rawIterator; + private AtomicInteger maxNumberOfAdditionalSplits; + private Lock lock; private final AtomicReference closeException = new AtomicReference<>(); private boolean closed; - private final boolean closeOnException; + private boolean closeOnException; + private List boundaryKeys; + private int boundIndex; + private KEY prefix; abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; + /** + * Retrieves a list of boundary keys based on the provided prefix and start key. + * These boundary keys can be used to split data into smaller ranges when processing. + * + * @param keyPrefix the prefix key that logically groups the keys of interest + * @param startKey the key from which to start retrieving boundary keys + * @return a list of byte arrays representing the boundary keys. + * @throws IOException if an I/O error occurs while retrieving the boundary keys + */ + abstract List getBoundaryKeys(KEY keyPrefix, KEY startKey) throws IOException; + + abstract int compare(RAW value1, byte[] value2); + abstract TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException; + KEY keyPrefix, KEY startKey, int maxParallelism) throws IOException; + + abstract Spliterator> createNewSpliterator(KEY prfx, byte[] startKey, int maxParallelism, + boolean closeOnEx, List boundKeys) throws IOException; RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - TableIterator> itr = getRawIterator(prefix, - startKey, maxParallelism); + List boundKeys = getBoundaryKeys(prefix, startKey).stream() + .sorted(ByteArrayCodec.getComparator()).collect(Collectors.toList()); + boundKeys.add(null); + init(prefix, startKey, maxParallelism, closeOnException, boundKeys); + } + + RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException, + List boundaryKeys) throws IOException { + init(prefix, startKey, maxParallelism, closeOnException, boundaryKeys); + } + + private void init(KEY prfx, KEY startKey, int maxParallelism, boolean closeOnEx, + List boundKeys) throws IOException { + TableIterator> itr = getRawIterator(prfx, startKey, maxParallelism); try { - this.closeOnException = closeOnException; + this.prefix = prfx; + this.boundaryKeys = boundKeys; + this.boundIndex = 0; + this.closeOnException = closeOnEx; this.lock = new ReentrantLock(); this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, @@ -84,6 +120,13 @@ public boolean tryAdvance(Consumer> action) { try { if (!closed && this.rawIterator.get().hasNext()) { kv = rawIterator.get().next(); + if (boundaryKeys.get(boundIndex) != null && compare(kv.getKey(), boundaryKeys.get(boundIndex)) >= 0) { + boundIndex++; + if (boundIndex >= boundaryKeys.size()) { + closeRawIterator(); + return false; + } + } } else { closeRawIterator(); return false; @@ -112,6 +155,24 @@ public int characteristics() { @Override public Spliterator> trySplit() { + if (boundIndex < boundaryKeys.size() - 1) { + lock.lock(); + try { + List boundaryKeysTmp = this.boundaryKeys; + int totalNumberOfElements = boundaryKeysTmp.size() - boundIndex; + if (totalNumberOfElements > 1) { + int splitIndex = boundIndex + totalNumberOfElements / 2; + this.boundaryKeys = boundaryKeysTmp.subList(0, splitIndex); + List nextSplitBoundaryKeys = boundaryKeysTmp.subList(splitIndex, boundaryKeysTmp.size()); + return createNewSpliterator(this.prefix, this.boundaryKeys.get(this.boundaryKeys.size() - 1), + maxNumberOfAdditionalSplits.get() + 1, closeOnException, nextSplitBoundaryKeys); + } + } catch (IOException ignored) { + // In case of exception, we can fall back to delegated deserialization for the spliterator. + } finally { + lock.unlock(); + } + } int val = maxNumberOfAdditionalSplits.decrementAndGet(); if (val >= 0) { try { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index bb150e6060dc..31fe912ce035 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -30,6 +30,9 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Spliterator; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.MetadataKeyFilters; import org.apache.hadoop.hdds.utils.TableCacheMetrics; @@ -44,6 +47,7 @@ import org.apache.hadoop.hdds.utils.db.cache.TableNoCache; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.function.CheckedBiFunction; +import org.rocksdb.LiveFileMetaData; /** * Strongly typed table implementation. @@ -54,7 +58,7 @@ * @param type of the keys in the store. * @param type of the values in the store. */ -public class TypedTable implements Table { +public class TypedTable implements BaseRDBTable { private static final long EPOCH_DEFAULT = -1L; static final int BUFFER_SIZE_DEFAULT = 4 << 10; // 4 KB @@ -548,6 +552,38 @@ TableCache getCache() { return cache; } + @Override + public List getTableSstFiles() throws IOException { + return rawTable.getTableSstFiles(); + } + + private List getBoundaryKeys(KEY prefix, KEY startKey) throws IOException { + return getTableSstFiles().stream() + .flatMap(liveFileMetaData -> Stream.of(liveFileMetaData.smallestKey(), liveFileMetaData.largestKey())) + .filter(value -> { + try { + byte[] prefixByteArray = encodeKey(prefix); + if (value.length < prefixByteArray.length) { + return false; + } + for (int i = 0; i < prefixByteArray.length; i++) { + if (value[i] != prefixByteArray[i]) { + return false; + } + } + } catch (IOException e) { + return false; + } + return true; + }).filter(value -> { + try { + return ByteArrayCodec.getComparator().compare(value, encodeKey(startKey)) >= 0; + } catch (IOException e) { + return false; + } + }).collect(Collectors.toList()); + } + /** * Key value implementation for strongly typed tables. */ @@ -602,67 +638,12 @@ KeyValue convert(RawKeyValue raw) private RawSpliterator newCodecBufferSpliterator( KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { - - @Override - KeyValue convert(RawKeyValue kv) throws IOException { - final int rawSize = kv.getValue().readableBytes(); - final KEY key = keyCodec.fromCodecBuffer(kv.getKey()); - final VALUE value = valueCodec.fromCodecBuffer(kv.getValue()); - return Table.newKeyValue(key, value, rawSize); - } - - @Override - TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException { - - CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); - CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); - try { - TableIterator> itr = - rawTable.iterator(prefixBuffer, maxParallelism); - if (startKeyBuffer != null) { - itr.seek(startKeyBuffer); - } - return itr; - } catch (Throwable t) { - if (prefixBuffer != null) { - prefixBuffer.release(); - } - throw t; - } finally { - if (startKeyBuffer != null) { - startKeyBuffer.release(); - } - } - } - }; + return new CodecBufferRawSpliterator(prefix, startKey, maxParallelism, closeOnException); } private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { - - @Override - KeyValue convert(RawKeyValue kv) throws IOException { - final int rawSize = kv.getValue().length; - final KEY key = keyCodec.fromPersistedFormat(kv.getKey()); - final VALUE value = valueCodec.fromPersistedFormat(kv.getValue()); - return Table.newKeyValue(key, value, rawSize); - } - - @Override - TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException { - byte[] prefixBytes = encodeKey(prefix); - byte[] startKeyBytes = encodeKey(startKey); - TableIterator> itr = rawTable.iterator(prefixBytes); - if (startKeyBytes != null) { - itr.seek(startKeyBytes); - } - return itr; - } - }; + return new ByteArrayRawSpliterator(prefix, startKey, maxParallelism, closeOnException); } /** @@ -751,4 +732,115 @@ public void removeFromDB() throws IOException { rawIterator.removeFromDB(); } } + + private final class CodecBufferRawSpliterator extends RawSpliterator { + + private CodecBufferRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + } + + private CodecBufferRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException, + List boundKeys) throws IOException { + super(prefix, startKey, maxParallelism, closeOnException, boundKeys); + } + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().readableBytes(); + final KEY key = keyCodec.fromCodecBuffer(kv.getKey()); + final VALUE value = valueCodec.fromCodecBuffer(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + List getBoundaryKeys(KEY prefix, KEY startKey) throws IOException { + return TypedTable.this.getBoundaryKeys(prefix, startKey); + } + + @Override + int compare(CodecBuffer value1, byte[] value2) { + return value1.compareTo(value2); + } + + @Override + TableIterator> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + + CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); + CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); + try { + TableIterator> itr = + rawTable.iterator(prefixBuffer, maxParallelism); + if (startKeyBuffer != null) { + itr.seek(startKeyBuffer); + } + return itr; + } catch (Throwable t) { + if (prefixBuffer != null) { + prefixBuffer.release(); + } + throw t; + } finally { + if (startKeyBuffer != null) { + startKeyBuffer.release(); + } + } + } + + @Override + Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, + boolean closeOnException, List boundaryKeys) throws IOException { + return new CodecBufferRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, boundaryKeys); + } + } + + private final class ByteArrayRawSpliterator extends RawSpliterator { + + private ByteArrayRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + } + + private ByteArrayRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException, + List boundKeys) throws IOException { + super(prefix, startKey, maxParallelism, closeOnException, boundKeys); + } + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().length; + final KEY key = keyCodec.fromPersistedFormat(kv.getKey()); + final VALUE value = valueCodec.fromPersistedFormat(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + List getBoundaryKeys(KEY prefix, KEY startKey) throws IOException { + return TypedTable.this.getBoundaryKeys(prefix, startKey); + } + + @Override + int compare(byte[] value1, byte[] value2) { + return ByteArrayCodec.getComparator().compare(value1, value2); + } + + @Override + TableIterator> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + byte[] prefixBytes = encodeKey(prefix); + byte[] startKeyBytes = encodeKey(startKey); + TableIterator> itr = rawTable.iterator(prefixBytes); + if (startKeyBytes != null) { + itr.seek(startKeyBytes); + } + return itr; + } + + @Override + Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, + boolean closeOnException, List boundaryKeys) throws IOException { + return new ByteArrayRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, boundaryKeys); + } + } } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java index 7b50cc8b0dc2..7b4e8ca03ea6 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java @@ -31,6 +31,8 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; import java.util.Spliterator; import java.util.function.Consumer; import org.junit.jupiter.api.Test; @@ -55,11 +57,27 @@ void testTryAdvanceWithValidElement() throws IOException { when(rawIteratorMock.next()).thenReturn(rawKeyValueMock); RawSpliterator rawSpliterator = new MockRawSpliterator(1) { + @Override + List getBoundaryKeys(String prefix, String startKey) { + return Collections.emptyList(); + } + + @Override + int compare(String value1, byte[] value2) { + return 0; + } + @Override TableIterator> getRawIterator(String prefix, String startKey, int maxParallelism) { return rawIteratorMock; } + + @Override + Spliterator> createNewSpliterator(String prefix, byte[] startKey, + int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { + return null; + } }; when(rawKeyValueMock.getKey()).thenReturn("key"); @@ -90,11 +108,27 @@ void testTryAdvanceWithNoElement() throws IOException { when(rawIteratorMock.hasNext()).thenReturn(false); RawSpliterator rawSpliterator = new MockRawSpliterator(1) { + @Override + List getBoundaryKeys(String prefix, String startKey) { + return Collections.emptyList(); + } + + @Override + int compare(String value1, byte[] value2) { + return 0; + } + @Override TableIterator> getRawIterator(String prefix, String startKey, int maxParallelism) { return rawIteratorMock; } + + @Override + Spliterator> createNewSpliterator(String prefix, byte[] startKey, + int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { + return null; + } }; Consumer> action = keyValue -> fail("Action should not be called"); @@ -124,11 +158,27 @@ Table.KeyValue convert(RawKeyValue kv) throws IOExceptio throw new IOException("Mocked exception"); } + @Override + List getBoundaryKeys(String prefix, String startKey) { + return Collections.emptyList(); + } + + @Override + int compare(String value1, byte[] value2) { + return 0; + } + @Override TableIterator> getRawIterator(String prefix, String startKey, int maxParallelism) { return rawIteratorMock; } + + @Override + Spliterator> createNewSpliterator(String prefix, byte[] startKey, + int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { + return null; + } }; Consumer> action = keyValue -> { @@ -145,16 +195,32 @@ TableIterator> } @Test - void testTrySplits() throws IOException { + void testTrySplitsWithNoBoundaryKeys() throws IOException { TableIterator> rawIteratorMock = mock(TableIterator.class); RawSpliterator rawSpliterator = new MockRawSpliterator(2) { + @Override + List getBoundaryKeys(String prefix, String startKey) { + return Collections.emptyList(); + } + + @Override + int compare(String value1, byte[] value2) { + return 0; + } + @Override TableIterator> getRawIterator(String prefix, String startKey, int maxParallelism) { return rawIteratorMock; } + + @Override + Spliterator> createNewSpliterator(String prefix, byte[] startKey, + int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { + return null; + } }; Spliterator> split1 = rawSpliterator.trySplit(); Spliterator> split2 = rawSpliterator.trySplit(); From 231ad287ba657b4345c6375480c0d3d548cb5e78 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 09:22:06 -0400 Subject: [PATCH 26/31] HDDS-12779. Revert creating an init function Change-Id: I2c32f89e4a9f65f46695bd9ab51ee75f78ebf2ab --- .../hadoop/hdds/utils/db/RawSpliterator.java | 38 +++++++++---------- 1 file changed, 18 insertions(+), 20 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index dece75293baf..1b36f0d26bd6 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -46,12 +46,12 @@ */ abstract class RawSpliterator implements Table.KeyValueSpliterator { - private ReferenceCountedObject>> rawIterator; - private AtomicInteger maxNumberOfAdditionalSplits; - private Lock lock; + private final ReferenceCountedObject>> rawIterator; + private final AtomicInteger maxNumberOfAdditionalSplits; + private final Lock lock; private final AtomicReference closeException = new AtomicReference<>(); private boolean closed; - private boolean closeOnException; + private final boolean closeOnException; private List boundaryKeys; private int boundIndex; private KEY prefix; @@ -72,31 +72,29 @@ abstract class RawSpliterator implements Table.KeyValueSplitera abstract int compare(RAW value1, byte[] value2); abstract TableIterator> getRawIterator( - KEY keyPrefix, KEY startKey, int maxParallelism) throws IOException; + KEY prefix, KEY startKey, int maxParallelism) throws IOException; - abstract Spliterator> createNewSpliterator(KEY prfx, byte[] startKey, int maxParallelism, - boolean closeOnEx, List boundKeys) throws IOException; + abstract Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, + boolean closeOnException, List boundaryKeys) throws IOException; RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - List boundKeys = getBoundaryKeys(prefix, startKey).stream() - .sorted(ByteArrayCodec.getComparator()).collect(Collectors.toList()); - boundKeys.add(null); - init(prefix, startKey, maxParallelism, closeOnException, boundKeys); + this(prefix, startKey, maxParallelism, closeOnException, null); } RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { - init(prefix, startKey, maxParallelism, closeOnException, boundaryKeys); - } - - private void init(KEY prfx, KEY startKey, int maxParallelism, boolean closeOnEx, - List boundKeys) throws IOException { - TableIterator> itr = getRawIterator(prfx, startKey, maxParallelism); + TableIterator> itr = getRawIterator(prefix, startKey, maxParallelism); try { - this.prefix = prfx; - this.boundaryKeys = boundKeys; + this.prefix = prefix; + if (boundaryKeys == null) { + this.boundaryKeys = getBoundaryKeys(prefix, startKey).stream().sorted(ByteArrayCodec.getComparator()).collect( + Collectors.toList()); + this.boundaryKeys.add(null); + } else { + this.boundaryKeys = boundaryKeys; + } this.boundIndex = 0; - this.closeOnException = closeOnEx; + this.closeOnException = closeOnException; this.lock = new ReentrantLock(); this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, From 78d33af98a53b2986571bf959666ee1f4cea209b Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 10:58:28 -0400 Subject: [PATCH 27/31] HDDS-12742. Convert to Named Class from anonymous class of RawSpliterator Change-Id: I73eb2d451d91d1f6177dc718f4b0cd2fbafb9d2c --- .../apache/hadoop/hdds/utils/db/RDBTable.java | 38 +++-- .../hadoop/hdds/utils/db/RawSpliterator.java | 26 +++- .../hadoop/hdds/utils/db/TypedTable.java | 130 ++++++++++-------- .../hdds/utils/db/RawSpliteratorTest.java | 3 +- 4 files changed, 118 insertions(+), 79 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index dc9a2857b416..cf4e37801468 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -377,23 +377,31 @@ && get(startKey) == null) { private RawSpliterator newByteArraySpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { + return new ByteArrayRawSpliterator(prefix, startKey, maxParallelism, closeOnException); + } - @Override - KeyValue convert(RawKeyValue kv) { - final int rawSize = kv.getValue().length; - return Table.newKeyValue(kv.getKey(), kv.getValue(), rawSize); - } + private class ByteArrayRawSpliterator extends RawSpliterator { - @Override - TableIterator> getRawIterator( - byte[] prefix, byte[] startKey, int maxParallelism) throws IOException { - TableIterator> itr = iterator(prefix); - if (startKey != null) { - itr.seek(startKey); - } - return itr; + public ByteArrayRawSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + initializeIterator(); + } + + @Override + KeyValue convert(RawKeyValue kv) { + final int rawSize = kv.getValue().length; + return Table.newKeyValue(kv.getKey(), kv.getValue(), rawSize); + } + + @Override + TableIterator> getRawIterator( + byte[] prefix, byte[] startKey, int maxParallelism) throws IOException { + TableIterator> itr = iterator(prefix); + if (startKey != null) { + itr.seek(startKey); } - }; + return itr; + } } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index 626dce515751..ce9451c8422c 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -44,25 +44,38 @@ */ abstract class RawSpliterator implements Table.KeyValueSpliterator { - private final ReferenceCountedObject>> rawIterator; + private ReferenceCountedObject>> rawIterator; + private final KEY prefix; + private final KEY startKey; private final AtomicInteger maxNumberOfAdditionalSplits; private final Lock lock; private final AtomicReference closeException = new AtomicReference<>(); private boolean closed; private final boolean closeOnException; + private boolean initialized; abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; abstract TableIterator> getRawIterator( KEY prefix, KEY startKey, int maxParallelism) throws IOException; - RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { + RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) { + this.prefix = prefix; + this.startKey = startKey; + this.closeOnException = closeOnException; + this.lock = new ReentrantLock(); + this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism); + this.initialized = false; + this.closed = false; + } + + synchronized void initializeIterator() throws IOException { + if (initialized) { + return; + } TableIterator> itr = getRawIterator(prefix, - startKey, maxParallelism); + startKey, maxNumberOfAdditionalSplits.decrementAndGet()); try { - this.closeOnException = closeOnException; - this.lock = new ReentrantLock(); - this.maxNumberOfAdditionalSplits = new AtomicInteger(maxParallelism - 1); this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, (completelyReleased) -> { if (completelyReleased) { @@ -75,6 +88,7 @@ abstract TableIterator> getRawIterator( itr.close(); throw e; } + initialized = true; } @Override diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index bb150e6060dc..ded020bba93d 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -602,67 +602,12 @@ KeyValue convert(RawKeyValue raw) private RawSpliterator newCodecBufferSpliterator( KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { - - @Override - KeyValue convert(RawKeyValue kv) throws IOException { - final int rawSize = kv.getValue().readableBytes(); - final KEY key = keyCodec.fromCodecBuffer(kv.getKey()); - final VALUE value = valueCodec.fromCodecBuffer(kv.getValue()); - return Table.newKeyValue(key, value, rawSize); - } - - @Override - TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException { - - CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); - CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); - try { - TableIterator> itr = - rawTable.iterator(prefixBuffer, maxParallelism); - if (startKeyBuffer != null) { - itr.seek(startKeyBuffer); - } - return itr; - } catch (Throwable t) { - if (prefixBuffer != null) { - prefixBuffer.release(); - } - throw t; - } finally { - if (startKeyBuffer != null) { - startKeyBuffer.release(); - } - } - } - }; + return new CodecBufferTypedRawSpliterator(prefix, startKey, maxParallelism, closeOnException); } private RawSpliterator newByteArraySpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { - return new RawSpliterator(prefix, startKey, maxParallelism, closeOnException) { - - @Override - KeyValue convert(RawKeyValue kv) throws IOException { - final int rawSize = kv.getValue().length; - final KEY key = keyCodec.fromPersistedFormat(kv.getKey()); - final VALUE value = valueCodec.fromPersistedFormat(kv.getValue()); - return Table.newKeyValue(key, value, rawSize); - } - - @Override - TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException { - byte[] prefixBytes = encodeKey(prefix); - byte[] startKeyBytes = encodeKey(startKey); - TableIterator> itr = rawTable.iterator(prefixBytes); - if (startKeyBytes != null) { - itr.seek(startKeyBytes); - } - return itr; - } - }; + return new ByteArrayTypedRawSpliterator(prefix, startKey, maxParallelism, closeOnException); } /** @@ -751,4 +696,75 @@ public void removeFromDB() throws IOException { rawIterator.removeFromDB(); } } + + private class CodecBufferTypedRawSpliterator extends RawSpliterator { + + public CodecBufferTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + initializeIterator(); + } + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().readableBytes(); + final KEY key = keyCodec.fromCodecBuffer(kv.getKey()); + final VALUE value = valueCodec.fromCodecBuffer(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + TableIterator> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + + CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix); + CodecBuffer startKeyBuffer = encodeKeyCodecBuffer(startKey); + try { + TableIterator> itr = + rawTable.iterator(prefixBuffer, maxParallelism); + if (startKeyBuffer != null) { + itr.seek(startKeyBuffer); + } + return itr; + } catch (Throwable t) { + if (prefixBuffer != null) { + prefixBuffer.release(); + } + throw t; + } finally { + if (startKeyBuffer != null) { + startKeyBuffer.release(); + } + } + } + } + + private class ByteArrayTypedRawSpliterator extends RawSpliterator { + + public ByteArrayTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + throws IOException { + super(prefix, startKey, maxParallelism, closeOnException); + initializeIterator(); + } + + @Override + KeyValue convert(RawKeyValue kv) throws IOException { + final int rawSize = kv.getValue().length; + final KEY key = keyCodec.fromPersistedFormat(kv.getKey()); + final VALUE value = valueCodec.fromPersistedFormat(kv.getValue()); + return Table.newKeyValue(key, value, rawSize); + } + + @Override + TableIterator> getRawIterator( + KEY prefix, KEY startKey, int maxParallelism) throws IOException { + byte[] prefixBytes = encodeKey(prefix); + byte[] startKeyBytes = encodeKey(startKey); + TableIterator> itr = rawTable.iterator(prefixBytes); + if (startKeyBytes != null) { + itr.seek(startKeyBytes); + } + return itr; + } + } } diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java index 7b50cc8b0dc2..2ae07fa1e868 100644 --- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java +++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/RawSpliteratorTest.java @@ -130,6 +130,7 @@ TableIterator> return rawIteratorMock; } }; + rawSpliterator.initializeIterator(); Consumer> action = keyValue -> { }; @@ -169,7 +170,7 @@ private abstract static class MockRawSpliterator extends RawSpliterator Date: Wed, 28 May 2025 11:05:01 -0400 Subject: [PATCH 28/31] HDDS-12742. Fix checkstyle Change-Id: Id594fc104fa32d69ab2a84ee5d08363af142e3d3 --- .../java/org/apache/hadoop/hdds/utils/db/RDBTable.java | 4 ++-- .../apache/hadoop/hdds/utils/db/RawSpliterator.java | 10 +++++----- .../org/apache/hadoop/hdds/utils/db/TypedTable.java | 8 ++++---- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java index cf4e37801468..fa61ff852920 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBTable.java @@ -380,9 +380,9 @@ private RawSpliterator newByteArraySpliterator(byte[] pr return new ByteArrayRawSpliterator(prefix, startKey, maxParallelism, closeOnException); } - private class ByteArrayRawSpliterator extends RawSpliterator { + private final class ByteArrayRawSpliterator extends RawSpliterator { - public ByteArrayRawSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) + private ByteArrayRawSpliterator(byte[] prefix, byte[] startKey, int maxParallelism, boolean closeOnException) throws IOException { super(prefix, startKey, maxParallelism, closeOnException); initializeIterator(); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index ce9451c8422c..1f4d3dd846c2 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -45,7 +45,7 @@ abstract class RawSpliterator implements Table.KeyValueSpliterator { private ReferenceCountedObject>> rawIterator; - private final KEY prefix; + private final KEY keyPrefix; private final KEY startKey; private final AtomicInteger maxNumberOfAdditionalSplits; private final Lock lock; @@ -57,10 +57,10 @@ abstract class RawSpliterator implements Table.KeyValueSplitera abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; abstract TableIterator> getRawIterator( - KEY prefix, KEY startKey, int maxParallelism) throws IOException; + KEY prefix, KEY start, int maxParallelism) throws IOException; - RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) { - this.prefix = prefix; + RawSpliterator(KEY keyPrefix, KEY startKey, int maxParallelism, boolean closeOnException) { + this.keyPrefix = keyPrefix; this.startKey = startKey; this.closeOnException = closeOnException; this.lock = new ReentrantLock(); @@ -73,7 +73,7 @@ synchronized void initializeIterator() throws IOException { if (initialized) { return; } - TableIterator> itr = getRawIterator(prefix, + TableIterator> itr = getRawIterator(keyPrefix, startKey, maxNumberOfAdditionalSplits.decrementAndGet()); try { this.rawIterator = ReferenceCountedObject.wrap(itr, () -> { }, diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index ded020bba93d..b714b212a820 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -697,9 +697,9 @@ public void removeFromDB() throws IOException { } } - private class CodecBufferTypedRawSpliterator extends RawSpliterator { + private final class CodecBufferTypedRawSpliterator extends RawSpliterator { - public CodecBufferTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + private CodecBufferTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { super(prefix, startKey, maxParallelism, closeOnException); initializeIterator(); @@ -739,9 +739,9 @@ TableIterator> getRawIterator } } - private class ByteArrayTypedRawSpliterator extends RawSpliterator { + private final class ByteArrayTypedRawSpliterator extends RawSpliterator { - public ByteArrayTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) + private ByteArrayTypedRawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { super(prefix, startKey, maxParallelism, closeOnException); initializeIterator(); From 18deb3cea95218d679b12c513936f77f559f2bcc Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 11:58:33 -0400 Subject: [PATCH 29/31] HDDS-12779. Fix checkstyle Change-Id: I04fe90cbfc65e4c52ed2b36c223ade76417c06c0 --- .../apache/hadoop/hdds/utils/db/RawSpliterator.java | 11 +++++------ .../org/apache/hadoop/hdds/utils/db/TypedTable.java | 6 ++++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index d0c6600df95b..0332f399269d 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -57,7 +57,6 @@ abstract class RawSpliterator implements Table.KeyValueSplitera private boolean initialized; private List boundaryKeys; private int boundIndex; - private KEY prefix; abstract Table.KeyValue convert(RawKeyValue kv) throws IOException; @@ -65,20 +64,20 @@ abstract class RawSpliterator implements Table.KeyValueSplitera * Retrieves a list of boundary keys based on the provided prefix and start key. * These boundary keys can be used to split data into smaller ranges when processing. * - * @param keyPrefix the prefix key that logically groups the keys of interest - * @param startKey the key from which to start retrieving boundary keys + * @param prefix the prefix key that logically groups the keys of interest + * @param start the key from which to start retrieving boundary keys * @return a list of byte arrays representing the boundary keys. * @throws IOException if an I/O error occurs while retrieving the boundary keys */ - abstract List getBoundaryKeys(KEY keyPrefix, KEY startKey) throws IOException; + abstract List getBoundaryKeys(KEY prefix, KEY start) throws IOException; abstract int compare(RAW value1, byte[] value2); abstract TableIterator> getRawIterator( KEY prefix, KEY start, int maxParallelism) throws IOException; - abstract Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, - boolean closeOnException, List boundaryKeys) throws IOException; + abstract Spliterator> createNewSpliterator(KEY prefix, byte[] start, int maxParallelism, + boolean closeOnEx, List boundKeys) throws IOException; RawSpliterator(KEY prefix, KEY startKey, int maxParallelism, boolean closeOnException) throws IOException { this(prefix, startKey, maxParallelism, closeOnException, null); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java index 7247811a6257..b60d0cc43d07 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java @@ -793,7 +793,8 @@ TableIterator> getRawIterator @Override Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { - return new CodecBufferRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, boundaryKeys); + return new CodecBufferTypedRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, + boundaryKeys); } } @@ -844,7 +845,8 @@ TableIterator> getRawIterator( @Override Spliterator> createNewSpliterator(KEY prefix, byte[] startKey, int maxParallelism, boolean closeOnException, List boundaryKeys) throws IOException { - return new ByteArrayRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, boundaryKeys); + return new ByteArrayTypedRawSpliterator(prefix, decodeKey(startKey), maxParallelism, closeOnException, + boundaryKeys); } } } From 82c3f013ab45d9167229b166e94f614c71248c55 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 12:11:44 -0400 Subject: [PATCH 30/31] HDDS-12779. Fix bug Change-Id: I65ca22255e2f0e5b9f58010a73fe53ac4cfe971e --- .../java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index 0332f399269d..09b794c1c763 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -93,7 +93,7 @@ abstract Spliterator> createNewSpliterator(KEY prefix this.initialized = false; this.closed = false; if (boundaryKeys == null) { - this.boundaryKeys = getBoundaryKeys(prefix, startKey).stream().sorted(ByteArrayCodec.getComparator()).collect( + this.boundaryKeys = getBoundaryKeys(keyPrefix, startKey).stream().sorted(ByteArrayCodec.getComparator()).collect( Collectors.toList()); this.boundaryKeys.add(null); } else { @@ -175,7 +175,7 @@ public Spliterator> trySplit() { int splitIndex = boundIndex + totalNumberOfElements / 2; this.boundaryKeys = boundaryKeysTmp.subList(0, splitIndex); List nextSplitBoundaryKeys = boundaryKeysTmp.subList(splitIndex, boundaryKeysTmp.size()); - return createNewSpliterator(this.prefix, this.boundaryKeys.get(this.boundaryKeys.size() - 1), + return createNewSpliterator(this.keyPrefix, this.boundaryKeys.get(this.boundaryKeys.size() - 1), maxNumberOfAdditionalSplits.get() + 1, closeOnException, nextSplitBoundaryKeys); } } catch (IOException ignored) { From a6c25e73429d5911be0cca93bac15b000e9b2eef Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 28 May 2025 13:36:01 -0400 Subject: [PATCH 31/31] HDDS-12779. Fix Boundary iteration logic Change-Id: I171b21260d363550ccd336b18042423055252547 --- .../java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java index 09b794c1c763..01955a30ea8c 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RawSpliterator.java @@ -131,7 +131,7 @@ public boolean tryAdvance(Consumer> action) { try { if (!closed && this.rawIterator.get().hasNext()) { kv = rawIterator.get().next(); - if (boundaryKeys.get(boundIndex) != null && compare(kv.getKey(), boundaryKeys.get(boundIndex)) >= 0) { + while (boundaryKeys.get(boundIndex) != null && compare(kv.getKey(), boundaryKeys.get(boundIndex)) >= 0) { boundIndex++; if (boundIndex >= boundaryKeys.size()) { closeRawIterator();