adoroszlai commented on code in PR #9542:
URL: https://github.com/apache/ozone/pull/9542#discussion_r2694516620
##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -1103,7 +1100,7 @@ public ListKeysResult listKeys(String volumeName, String
bucketName,
long readFromRDbStartNs, readFromRDbStopNs = 0;
// Get maxKeys from DB if it has.
try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>>
- keyIter = getKeyTable(getBucketLayout()).iterator()) {
+ keyIter = getKeyTable(getBucketLayout()).iterator(null, seekKey))
{
readFromRDbStartNs = Time.monotonicNowNanos();
KeyValue< String, OmKeyInfo > kv;
keyIter.seek(seekKey);
Review Comment:
Isn't this `seek` now unnecessary?
##########
hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/db/TestRDBStore.java:
##########
@@ -403,4 +409,136 @@ private void compareSstWithSameName(File checkpoint1,
File checkpoint2)
}
}
}
+
+ @Test
+ public void testIteratorWithSeek() throws Exception {
+ final Table<byte[], byte[]> table = rdbStore.getTable(families.get(0));
+ // Write keys: a1, a3, a5, b2, b4
+ table.put(getBytesUtf16("a1"), getBytesUtf16("val1"));
+ table.put(getBytesUtf16("a3"), getBytesUtf16("val3"));
+ table.put(getBytesUtf16("a5"), getBytesUtf16("val5"));
+ table.put(getBytesUtf16("b2"), getBytesUtf16("val2"));
+ table.put(getBytesUtf16("b4"), getBytesUtf16("val4"));
+
+ // Case 1: Seek to existing key, no prefix
+ try (TableIterator<byte[], ? extends Table.KeyValue<byte[], byte[]>> iter
= table.iterator(null,
+ getBytesUtf16("a3"))) {
+ assertTrue(iter.hasNext());
+ assertArrayEquals(getBytesUtf16("a3"), iter.next().getKey());
+ assertTrue(iter.hasNext());
+ assertArrayEquals(getBytesUtf16("a5"), iter.next().getKey());
Review Comment:
I'd like to suggest adding a helper method to verify the list of keys
iterated. Something like:
```java
private static void assertIterator(String prefix, String seek,
Table<byte[], ?> table, List<String> expected)
throws RocksDatabaseException, CodecException {
final byte[] prefixBytes = prefix != null ? getBytesUtf16(prefix) : null;
final byte[] seekBytes = seek != null ? getBytesUtf16(seek) : null;
try (TableIterator<byte[], ? extends Table.KeyValue<byte[], ?>> iter =
table.iterator(prefixBytes, seekBytes)) {
for (String e : expected) {
assertTrue(iter.hasNext());
assertArrayEquals(getBytesUtf16(e), iter.next().getKey());
}
assertFalse(iter.hasNext());
}
}
```
Then we can both simplify cases and make sure the iterator stops (like
already done in two of the five).
```java
// Case 1: Seek to existing key, no prefix
assertIterator(null, "a3", table, asList("a3", "a5", "b2", "b4"));
// Case 2: Seek to non-existent key (should land on next greater), no
prefix
assertIterator(null, "a2", table, asList("a3", "a5", "b2", "b4"));
// Case 3: Seek past all keys, no prefix
assertIterator(null, "z9", table, emptyList());
// Case 4: Seek with prefix
assertIterator("b", "b3", table, singletonList("b4"));
// Case 5: Seek with prefix to start of prefix
assertIterator("b", "b2", table, asList("b2", "b4"));
```
It can be reused in `testIteratorSeekEdgeCases`, too.
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java:
##########
@@ -29,9 +29,18 @@ private static byte[] copyPrefix(byte[] prefix) {
}
RDBStoreByteArrayIterator(ManagedRocksIterator iterator,
- RDBTable table, byte[] prefix, IteratorType type) {
+ RDBTable table, byte[] prefix, IteratorType type) {
+ this(iterator, table, prefix, type, null);
+ }
+
+ RDBStoreByteArrayIterator(ManagedRocksIterator iterator,
+ RDBTable table, byte[] prefix, IteratorType type,
byte[] seekKey) {
Review Comment:
nit: Please do not format method signature like this. Whenever visibility /
return type / method name / other modifiers are changed, we would have to
reindent all parameters.
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreByteArrayIterator.java:
##########
@@ -29,9 +29,18 @@ private static byte[] copyPrefix(byte[] prefix) {
}
RDBStoreByteArrayIterator(ManagedRocksIterator iterator,
- RDBTable table, byte[] prefix, IteratorType type) {
+ RDBTable table, byte[] prefix, IteratorType type) {
Review Comment:
```suggestion
RDBTable table, byte[] prefix, IteratorType type) {
```
nit: Please disable `Align when multiline` in settings (if using IDEA).
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TypedTable.java:
##########
@@ -573,7 +587,9 @@ abstract class RawIterator<RAW>
this.rawIterator = rawIterator;
}
- /** Covert the given key to the {@link RAW} type. */
+ /**
+ * Covert the given key to the {@link RAW} type.
+ */
Review Comment:
nit: let's not change this unnecessarily
```suggestion
/** Covert the given key to the {@link RAW} type. */
```
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/Table.java:
##########
@@ -146,7 +146,7 @@ default VALUE getReadCopy(KEY key) throws
RocksDatabaseException, CodecException
* Deletes a range of keys from the metadata store.
*
* @param beginKey start metadata key
- * @param endKey end metadata key
+ * @param endKey end metadata key
Review Comment:
```suggestion
* @param endKey end metadata key
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]