m-trieu commented on code in PR #29401:
URL: https://github.com/apache/beam/pull/29401#discussion_r1394954158


##########
sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Cache.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.beam.io.requestresponse;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteSource;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+
+/** Transforms for reading and writing request/response associations to a 
cache. */
+class Cache {

Review Comment:
   does this class not have members and is only for using the static methods? 
if yes recommend making the class final `final class Cache` and having a 
private constructor to make that behavior/interface explicit.



##########
sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Cache.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.beam.io.requestresponse;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteSource;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+
+/** Transforms for reading and writing request/response associations to a 
cache. */
+class Cache {
+
+  /**
+   * Instantiates a {@link Call} {@link PTransform} that reads {@link 
RequestT} {@link ResponseT}
+   * associations from a cache. The {@link KV} value is null when no 
association exists. This method
+   * does not enforce {@link Coder#verifyDeterministic} and defers to the user 
to determine whether
+   * to enforce this given the cache implementation.
+   */
+  static <
+          @NonNull RequestT,
+          @Nullable ResponseT,
+          CallerSetupTeardownT extends
+              @NonNull Caller<@NonNull RequestT, KV<@NonNull RequestT, 
@Nullable ResponseT>>
+                  & SetupTeardown>
+      PTransform<
+              @NonNull PCollection<@NonNull RequestT>,
+              Call.@NonNull Result<KV<@NonNull RequestT, @Nullable ResponseT>>>
+          read(
+              @NonNull CallerSetupTeardownT implementsCallerSetupTeardown,
+              @NonNull Coder<@NonNull RequestT> requestTCoder,
+              @NonNull Coder<@Nullable ResponseT> responseTCoder) {
+    return Call.ofCallerAndSetupTeardown(
+        implementsCallerSetupTeardown, KvCoder.of(requestTCoder, 
responseTCoder));
+  }
+
+  /**
+   * Instantiates a {@link Call} {@link PTransform}, calling {@link #read} 
with a {@link Caller}
+   * that employs a redis client.
+   *
+   * <p>This method requires both the {@link RequestT} and {@link ResponseT}s' 
{@link
+   * Coder#verifyDeterministic}. Otherwise, it throws a {@link 
NonDeterministicException}.
+   *
+   * <p><a href="https://redis.io";>Redis</a> is designed for multiple 
workloads, simultaneously
+   * reading and writing to a shared instance. See <a
+   * href="https://redis.io/docs/get-started/faq/";>Redis FAQ</a> for more 
information on important
+   * considerations when using this method to achieve cache reads.
+   */
+  static <@NonNull RequestT, @Nullable ResponseT>
+      PTransform<
+              @NonNull PCollection<@NonNull RequestT>,
+              Call.@NonNull Result<KV<@NonNull RequestT, @Nullable ResponseT>>>
+          readUsingRedis(
+              @NonNull RedisClient client,
+              @NonNull Coder<@NonNull RequestT> requestTCoder,
+              @NonNull Coder<@Nullable ResponseT> responseTCoder)
+              throws NonDeterministicException {
+    return read(
+        new UsingRedis<>(requestTCoder, responseTCoder, client).read(),
+        requestTCoder,
+        responseTCoder);
+  }
+
+  /**
+   * Write a {@link RequestT} {@link ResponseT} association to a cache. This 
method does not enforce
+   * {@link Coder#verifyDeterministic} and defers to the user to determine 
whether to enforce this
+   * given the cache implementation.
+   */
+  static <
+          @NonNull RequestT,

Review Comment:
   they are NonNull by default so feel free to remove.  I think it also 
improves readability if we default by assuming nonnull and forcing nullability 
to be expressed by the `@Nullable` annotation or (preferred if possible) 
`Optional<T>` 



##########
sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/RedisClient.java:
##########
@@ -61,6 +62,15 @@ long decr(String key) throws UserCodeExecutionException {
     }
   }
 
+  /** Get a byte array associated with a byte array key. Returns null if key 
does not exist. */
+  byte @Nullable [] getBytes(byte[] key) throws UserCodeExecutionException {

Review Comment:
   can this just return an empty array?
   or an Optional?



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