lhotari commented on code in PR #21966:
URL: https://github.com/apache/pulsar/pull/21966#discussion_r1466023980


##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/PulsarMetadataStateStoreImpl.java:
##########
@@ -111,6 +112,20 @@ public CompletableFuture<ByteBuffer> getAsync(String key) {
                                 .orElse(null));
     }
 
+    @Override
+    public StateValue getStateValue(String key) {
+        return getStateValueAsync(key).join();
+    }
+
+    @Override
+    public CompletableFuture<StateValue> getStateValueAsync(String key) {
+        return store.get(getPath(key))
+                .thenApply(optRes ->
+                        optRes.map(x ->
+                            new StateValue(ByteBuffer.wrap(x.getValue()), 
x.getStat().getVersion(), null))

Review Comment:
   Is there a benefit of `ByteBuffer.wrap` when it's immediately accessed 
directly in `StateValue` constructor (in the questionable way)? Perhaps it 
would be better to pass what `byte[]` directly?



##########
pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/state/StateValue.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.pulsar.functions.api.state;
+
+import java.nio.ByteBuffer;
+
+public class StateValue {
+    private final ByteBuffer value;
+    private final Long version;
+    private final Boolean isNumber;
+
+    public StateValue(ByteBuffer value, Long version, Boolean isNumber) {
+        this.value = value == null ? null : ByteBuffer.wrap(value.array());

Review Comment:
   The javadoc of `ByteBuffer.array` method says that the method should only be 
called if `hasArray()` returns true.
   The underlying array might have a specific position and limit. I think that 
`ByteBuffer.wrap(value.array())` is not a very good approach for duplicating a 
ByteBuffer.



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreImpl.java:
##########
@@ -190,4 +191,39 @@ public ByteBuffer get(String key) {
             throw new RuntimeException("Failed to retrieve the state value for 
key '" + key + "'", e);
         }
     }
+
+    @Override
+    public StateValue getStateValue(String key) {
+        try {
+            return result(getStateValueAsync(key));
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to retrieve the state value for 
key '" + key + "'", e);
+        }
+    }
+
+    @Override
+    public CompletableFuture<StateValue> getStateValueAsync(String key) {
+        return 
table.getKv(Unpooled.wrappedBuffer(key.getBytes(UTF_8))).thenApply(
+                data -> {
+                    try {
+                        if (data != null && data.value() != null) {
+                            ByteBuffer result = 
ByteBuffer.allocate(data.value().readableBytes());
+                            data.value().readBytes(result);
+                            // Set position to off the buffer to the 
beginning, since the position after the
+                            // read is going to be end of the buffer
+                            // If we do not rewind to the beginning here, 
users will have to explicitly do
+                            // this in their function code
+                            // in order to use any of the ByteBuffer operations
+                            result.position(0);
+                            return new StateValue(result, data.version(), 
data.isNumber());
+                        }
+                        return null;
+                    } finally {
+                        if (data != null) {
+                            ReferenceCountUtil.safeRelease(data);

Review Comment:
   This doesn't make sense since `ReferenceCountUtil` is used for Netty's 
`ByteBuf`. In this case, we are using `java.nio.ByteBuffer`.



##########
pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/state/BKStateStoreImpl.java:
##########
@@ -190,4 +191,39 @@ public ByteBuffer get(String key) {
             throw new RuntimeException("Failed to retrieve the state value for 
key '" + key + "'", e);
         }
     }
+
+    @Override
+    public StateValue getStateValue(String key) {
+        try {
+            return result(getStateValueAsync(key));
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to retrieve the state value for 
key '" + key + "'", e);
+        }
+    }
+
+    @Override
+    public CompletableFuture<StateValue> getStateValueAsync(String key) {
+        return 
table.getKv(Unpooled.wrappedBuffer(key.getBytes(UTF_8))).thenApply(
+                data -> {
+                    try {
+                        if (data != null && data.value() != null) {
+                            ByteBuffer result = 
ByteBuffer.allocate(data.value().readableBytes());
+                            data.value().readBytes(result);
+                            // Set position to off the buffer to the 
beginning, since the position after the
+                            // read is going to be end of the buffer
+                            // If we do not rewind to the beginning here, 
users will have to explicitly do
+                            // this in their function code
+                            // in order to use any of the ByteBuffer operations
+                            result.position(0);

Review Comment:
   isn't this making the assumption that position was 0 before reading? Aren't 
ByteBuffer's `mark` and `reset` meant to be used for this type of use cases?



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

Reply via email to