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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -102,7 +105,8 @@ private StreamingEngineClient(
       WindmillStubFactory stubFactory,
       GetWorkBudgetDistributor getWorkBudgetDistributor,
       GrpcDispatcherClient dispatcherClient,
-      long clientId) {
+      long clientId,
+      ChannelCache channelCache) {

Review Comment:
   It seems like the channelcache is overlapping with the stubfactory.
   Maybe it would be better to merge them by taking in a 
ChannelCachingRemoteStubFactory?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -468,10 +472,18 @@ private StreamingDataflowWorker(
   public static StreamingDataflowWorker 
fromOptions(DataflowWorkerHarnessOptions options) {
     ConcurrentMap<String, ComputationState> computationMap = new 
ConcurrentHashMap<>();
     long clientId = clientIdGenerator.nextLong();
+    ChannelCache channelCache =

Review Comment:
   can we move the channel cache usage to 
StreamingEngineClient/GrpcWindmillServer and below?
   
   Seems like unnecessary to plumb from up here and given that we are 
coordinating invalidation at that level it doesn't seem like we want to expose 
it here.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.runners.dataflow.worker.windmill.client.grpc.stubs;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import java.io.PrintWriter;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress;
+import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Cache of gRPC channels for {@link WindmillServiceAddress}. Follows <a
+ * href=https://grpc.io/docs/guides/performance/#java>gRPC recommendations</a> 
for re-using channels
+ * when possible.
+ *
+ * @implNote Backed by {@link LoadingCache} which is thread-safe.
+ */
+@ThreadSafe
+public final class ChannelCache implements StatusDataProvider {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ChannelCache.class);
+  private final LoadingCache<WindmillServiceAddress, ManagedChannel> 
channelCache;
+
+  private ChannelCache(
+      boolean useIsolatedChannels,
+      Function<WindmillServiceAddress, ManagedChannel> channelFactory,
+      RemovalListener<WindmillServiceAddress, ManagedChannel> 
onChannelRemoved) {
+    this.channelCache =
+        Caffeine.newBuilder()
+            .removalListener(onChannelRemoved)
+            .build(
+                serviceAddress ->
+                    // IsolationChannel will create and manage separate RPC 
channels to the same
+                    // serviceAddress via calling the channelFactory, else 
just directly return the
+                    // RPC channel.
+                    useIsolatedChannels

Review Comment:
   it doesn't seem like IsolationChannel needs to be part of the cache
   callers can just pass in a channelFactory that vends IsolationChannels if 
desired.



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