Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -41,11 +41,17 @@ abstract class RDBStoreAbstractIterator<RAW>
// prefix for each key.
private final RAW prefix;

RDBStoreAbstractIterator(ManagedRocksIterator iterator, RDBTable table,
RAW prefix) {
private final Type type;

RDBStoreAbstractIterator(ManagedRocksIterator iterator, RDBTable table, RAW prefix, Type type) {
this.rocksDBIterator = iterator;
this.rocksDBTable = table;
this.prefix = prefix;
this.type = this.prefix == null ? type : type.addKey(); // it has to read key for matching prefix.
}

Type getType() {
return type;
}

/** @return the key for the current entry. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,13 @@
class RDBStoreByteArrayIterator extends RDBStoreAbstractIterator<byte[]> {
private static final byte[] EMPTY = {};

private final Type type;
private static byte[] copyPrefix(byte[] prefix) {
return prefix == null || prefix.length == 0 ? null : Arrays.copyOf(prefix, prefix.length);
}

RDBStoreByteArrayIterator(ManagedRocksIterator iterator,
RDBTable table, byte[] prefix, Type type) {
super(iterator, table,
prefix == null ? null : Arrays.copyOf(prefix, prefix.length));
this.type = type;
super(iterator, table, copyPrefix(prefix), type);
seekToFirst();
}

Expand All @@ -44,8 +44,8 @@ byte[] key() {
@Override
Table.KeyValue<byte[], byte[]> getKeyValue() {
final ManagedRocksIterator i = getRocksDBIterator();
final byte[] key = type.readKey() ? i.get().key() : EMPTY;
final byte[] value = type.readValue() ? i.get().value() : EMPTY;
final byte[] key = getType().readKey() ? i.get().key() : EMPTY;
final byte[] value = getType().readValue() ? i.get().value() : EMPTY;
return Table.newKeyValue(key, value);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,15 +33,15 @@ class RDBStoreCodecBufferIterator extends RDBStoreAbstractIterator<CodecBuffer>

RDBStoreCodecBufferIterator(ManagedRocksIterator iterator, RDBTable table,
CodecBuffer prefix, Type type) {
super(iterator, table, prefix);
super(iterator, table, prefix, type);

final String name = table != null ? table.getName() : null;
this.keyBuffer = new Buffer(
new CodecBuffer.Capacity(name + "-iterator-key", 1 << 10),
type.readKey() ? buffer -> getRocksDBIterator().get().key(buffer) : null);
getType().readKey() ? buffer -> getRocksDBIterator().get().key(buffer) : null);
this.valueBuffer = new Buffer(
new CodecBuffer.Capacity(name + "-iterator-value", 4 << 10),
type.readValue() ? buffer -> getRocksDBIterator().get().value(buffer) : null);
getType().readValue() ? buffer -> getRocksDBIterator().get().value(buffer) : null);
seekToFirst();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,9 @@ default KeyValueIterator<KEY, VALUE> iterator(KeyValueIterator.Type type)
*
* @param prefix The prefix of the elements to be iterated.
* @param type Specify whether key and/or value are required.
* When the prefix is non-empty, it has to read keys for matching the prefix.
* The type will be automatically changed to including keys;
* see {@link KeyValueIterator.Type#addKey()}.
* @return an iterator.
*/
KeyValueIterator<KEY, VALUE> iterator(KEY prefix, KeyValueIterator.Type type)
Expand Down Expand Up @@ -392,6 +395,10 @@ boolean readKey() {
boolean readValue() {
return (this.ordinal() & VALUE_ONLY.ordinal()) != 0;
}

Type addKey() {
return values()[ordinal() | KEY_ONLY.ordinal()];
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -389,15 +389,7 @@ public void deleteRange(KEY beginKey, KEY endKey) throws RocksDatabaseException,
public KeyValueIterator<KEY, VALUE> iterator(KEY prefix, KeyValueIterator.Type type)
throws RocksDatabaseException, CodecException {
if (supportCodecBuffer) {
final CodecBuffer prefixBuffer = encodeKeyCodecBuffer(prefix);
try {
return newCodecBufferTableIterator(rawTable.iterator(prefixBuffer, type));
} catch (Throwable t) {
if (prefixBuffer != null) {
prefixBuffer.release();
}
throw t;
}
return newCodecBufferTableIterator(prefix, type);
} else {
final byte[] prefixBytes = encodeKey(prefix);
return new TypedTableIterator(rawTable.iterator(prefixBytes, type));
Expand Down Expand Up @@ -516,8 +508,28 @@ TableCache<KEY, VALUE> getCache() {
return cache;
}

RawIterator<CodecBuffer> newCodecBufferTableIterator(
KeyValueIterator<CodecBuffer, CodecBuffer> i) {
private RawIterator<CodecBuffer> newCodecBufferTableIterator(KEY prefix, KeyValueIterator.Type type)
throws RocksDatabaseException, CodecException {
final CodecBuffer encoded = encodeKeyCodecBuffer(prefix);
final CodecBuffer prefixBuffer;
if (encoded != null && encoded.readableBytes() == 0) {
encoded.release();
prefixBuffer = null;
} else {
prefixBuffer = encoded;
}

try {
return newCodecBufferTableIterator(rawTable.iterator(prefixBuffer, type));
} catch (Throwable t) {
if (prefixBuffer != null) {
prefixBuffer.release();
}
throw t;
}
}

private RawIterator<CodecBuffer> newCodecBufferTableIterator(KeyValueIterator<CodecBuffer, CodecBuffer> i) {
return new RawIterator<CodecBuffer>(i) {
@Override
AutoCloseSupplier<CodecBuffer> convert(KEY key) throws CodecException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.apache.hadoop.hdds.utils.db.Table.KeyValueIterator.Type.NEITHER;
import static org.apache.hadoop.hdds.utils.db.Table.KeyValueIterator.Type.VALUE_ONLY;
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;
Expand Down Expand Up @@ -315,5 +316,10 @@ public void testIteratorType() {

assertTrue(KEY_AND_VALUE.readKey());
assertTrue(KEY_AND_VALUE.readValue());

assertEquals(KEY_ONLY, NEITHER.addKey());
assertEquals(KEY_ONLY, KEY_ONLY.addKey());
assertEquals(KEY_AND_VALUE, VALUE_ONLY.addKey());
assertEquals(KEY_AND_VALUE, KEY_AND_VALUE.addKey());
}
}
Loading