szetszwo commented on code in PR #8203:
URL: https://github.com/apache/ozone/pull/8203#discussion_r2045242409


##########
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;
+  private final Set<RawKeyValue<Buffer>> inUseBuffers;
+  private final Object bufferLock;

Review Comment:
   `CodecBuffer` already uses a netty buffer pool.  The buffer stack is not 
needed.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreAbstractIterator.java:
##########
@@ -78,38 +82,50 @@ final RAW getPrefix() {
 
   @Override
   public final void forEachRemaining(
-      Consumer<? super Table.KeyValue<RAW, RAW>> action) {
+      Consumer<? super UncheckedAutoCloseableSupplier<RawKeyValue<RAW>>> 
action) {
     while (hasNext()) {
-      action.accept(next());
+      UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> entry = next();
+      action.accept(entry);
     }
   }
 
   private void setCurrentEntry() {
-    if (rocksDBIterator.get().isValid()) {
+    if (currentEntry != null) {
+      currentEntry.release();
+    }
+
+    boolean isValid = !closed && rocksDBIterator.get().isValid();
+    if (isValid) {
       currentEntry = getKeyValue();
+      currentEntry.retain();
     } else {
       currentEntry = null;
     }
+    setHasNext(isValid, currentEntry);
+  }
+
+  public void setHasNext(boolean isValid, 
ReferenceCountedObject<RawKeyValue<RAW>> entry) {
+    this.hasNext = isValid && (prefix == null || 
startsWithPrefix(entry.get().getKey()));
   }
 
   @Override
   public final boolean hasNext() {
-    return rocksDBIterator.get().isValid() &&
-        (prefix == null || startsWithPrefix(key()));
+    return hasNext;
   }
 
   @Override
-  public final Table.KeyValue<RAW, RAW> next() {
-    setCurrentEntry();
-    if (currentEntry != null) {
+  public final synchronized UncheckedAutoCloseableSupplier<RawKeyValue<RAW>> 
next() {

Review Comment:
   Why it needs `synchronized`?  Will an iterator object be invoked by multiple 
threads?



-- 
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]

Reply via email to