szetszwo commented on code in PR #8203:
URL: https://github.com/apache/ozone/pull/8203#discussion_r2047786815
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreCodecBufferIterator.java:
##########
@@ -84,38 +92,77 @@ CodecBuffer getFromDb() {
}
}
- private final Buffer keyBuffer;
- private final Buffer valueBuffer;
+ private final Stack<RawKeyValue<Buffer>> availableBufferStack;
Review Comment:
Stack is not the best data structure in this case. We may use a
[BlockingQueue](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/BlockingQueue.html).
It is even better to use `ThreadLoacl<Buffer>` as mentioned my previous
comment.
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java:
##########
@@ -78,32 +82,44 @@ final RAW getPrefix() {
@Override
public final void forEachRemaining(
- Consumer<? super Table.KeyValue<RAW, RAW>> action) {
+ Consumer<? super UncheckedAutoCloseableSupplier<RawKeyValue<RAW>>>
action) {
while (hasNext()) {
- action.accept(next());
+ UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> entry = next();
+ action.accept(entry);
}
}
private void setCurrentEntry() {
- if (rocksDBIterator.get().isValid()) {
+ if (currentEntry != null) {
+ currentEntry.release();
+ }
+
+ boolean isValid = !closed && rocksDBIterator.get().isValid();
+ if (isValid) {
currentEntry = getKeyValue();
+ currentEntry.retain();
} else {
currentEntry = null;
}
+ setHasNext(isValid, currentEntry);
+ }
+
+ public void setHasNext(boolean isValid,
ReferenceCountedObject<RawKeyValue<RAW>> entry) {
+ this.hasNext = isValid && (prefix == null ||
startsWithPrefix(entry.get().getKey()));
}
@Override
public final boolean hasNext() {
- return rocksDBIterator.get().isValid() &&
- (prefix == null || startsWithPrefix(key()));
+ return hasNext;
}
@Override
- public final Table.KeyValue<RAW, RAW> next() {
- setCurrentEntry();
- if (currentEntry != null) {
+ public final UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> next() {
Review Comment:
@swamirishi , The current single-threaded API is below
```java
TypedTable<ContainerID, String> table = ...;
try (TableIterator<ContainerID, ? extends Table.KeyValue<ContainerID,
String>> i = table.iterator()) {
final KeyValue<ContainerID, String> keyValue = i.next();
// process it
}
```
What is the multi-threaded API in your mind? What would the above code look
like?
--
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]