scwhittle commented on code in PR #33566:
URL: https://github.com/apache/beam/pull/33566#discussion_r1912906834


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/DetectNewPartitionsDoFn.java:
##########
@@ -60,6 +62,7 @@ public class DetectNewPartitionsDoFn extends 
DoFn<PartitionMetadata, PartitionMe
   private final MapperFactory mapperFactory;
   private final ActionFactory actionFactory;
   private final ChangeStreamMetrics metrics;
+  private final CacheFactory cacheFactory;

Review Comment:
   I'm not sure that this will result in multiple DoFn instances sharing the 
same cache.
   Each DoFn is deserialized separately, so I think that they would each have 
their own CacheFactory instance, and then each one would get a separate cache.
   
   You could make it static but then the cache woudl be shared against possibly 
different instances of DetectNewPartitionsDoFn in the graph (which maybe could 
have different configuration).
   
   For Kafak reads, Stephen was using this approach:
   ```
       private static final Map<Long, StaticState> StaticStateCache = new 
ConcurrentHashMap<>();
   
     private static final AtomicLong FN_ID = new AtomicLong();
   
     // A unique identifier for the instance. Generally unique unless the ID 
generator overflows.
     private final long fnId = FN_ID.getAndIncrement();
   ```
   
   Then within a graph you might have mulitple of the DoFn with unique fnId 
variables. When deserialized, each of the deserialized instances will have the 
fnId set to the same value. Then the static state cache can be used to have the 
same state for a given node in the graph across all of the deserialized 
instances.
   
   Since this is tricky to get right, it seems like perhaps we could come up 
with some annotation or other helper class to do this.
   
   
   



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/cache/LoadingWatermarkCache.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.sdk.io.gcp.spanner.changestreams.cache;
+
+import com.google.cloud.Timestamp;
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import javax.annotation.Nullable;
+import 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataDao;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache;
+
+public class LoadingWatermarkCache implements WatermarkCache {
+  private static final Object MIN_WATERMARK_KEY = new Object();
+
+  private final LoadingCache<Object, Optional<Timestamp>> cache;
+
+  public LoadingWatermarkCache(PartitionMetadataDao dao, 
org.joda.time.Duration refreshRate) {
+    this.cache =
+        CacheBuilder.newBuilder()
+            .refreshAfterWrite(Duration.ofMillis(refreshRate.getMillis()))

Review Comment:
   think you need to make the loading async
   
   I think you can use this method:
   
https://guava.dev/releases/22.0/api/docs/com/google/common/cache/CacheLoader.html#asyncReloading-com.google.common.cache.CacheLoader-java.util.concurrent.Executor-



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/cache/CacheFactory.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sdk.io.gcp.spanner.changestreams.cache;
+
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.DaoFactory;
+import org.joda.time.Duration;
+
+@SuppressWarnings({"initialization.fields.uninitialized", "nullness"})
+public class CacheFactory implements Serializable {
+
+  private static final long serialVersionUID = -8722905670370252723L;
+  private final DaoFactory daoFactory;
+  private final @Nullable Duration refreshRate;
+  private transient WatermarkCache watermarkCacheInstance;
+
+  public CacheFactory(DaoFactory daoFactory, @Nullable Duration 
watermarkRefreshRate) {
+    this.daoFactory = daoFactory;
+    this.refreshRate = watermarkRefreshRate;
+  }
+
+  public synchronized WatermarkCache getWatermarkCache() {
+    if (watermarkCacheInstance == null) {
+      if (refreshRate == null) {
+        watermarkCacheInstance = new 
NoOpWatermarkCache(daoFactory.getPartitionMetadataDao());

Review Comment:
   alternatively you could set this in the constructor and in readObject
   
   
https://stackoverflow.com/questions/4119181/java-object-serialization-readobject-defaultreadobject



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/cache/CacheFactory.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.sdk.io.gcp.spanner.changestreams.cache;
+
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.DaoFactory;
+import org.joda.time.Duration;
+
+@SuppressWarnings({"initialization.fields.uninitialized", "nullness"})

Review Comment:
   can you fix these instead of suppressing? Can watermarkCacheInstance just be 
marked nullable, I think it might fix both



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