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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ConfigAwareChannelFactory.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 static 
org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannels.remoteChannel;
+
+import javax.annotation.concurrent.ThreadSafe;
+import 
org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress;
+import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Internal;
+import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
+
+/** Creates gRPC channels based on the current {@link StreamingGlobalConfig}. 
*/
+@Internal
+@ThreadSafe
+public class ConfigAwareChannelFactory implements WindmillChannelFactory {
+  private final int windmillServiceRpcChannelAliveTimeoutSec;
+  private final boolean isIsolationChannelsEnabledByDefault;
+  @MonotonicNonNull private StreamingGlobalConfig currentConfig = null;
+
+  public ConfigAwareChannelFactory(
+      int windmillServiceRpcChannelAliveTimeoutSec, boolean 
isIsolationChannelsEnabledByDefault) {
+    this.windmillServiceRpcChannelAliveTimeoutSec = 
windmillServiceRpcChannelAliveTimeoutSec;
+    this.isIsolationChannelsEnabledByDefault = 
isIsolationChannelsEnabledByDefault;
+  }
+
+  @Override
+  public synchronized ManagedChannel create(
+      Windmill.UserWorkerGrpcFlowControlSettings flowControlSettings,
+      WindmillServiceAddress serviceAddress) {
+    return isIsolationChannelsEnabled()
+        // IsolationChannel will create and manage separate RPC channels to 
the same
+        // serviceAddress via calling the channelFactory, else just directly 
return
+        // the RPC channel.
+        ? IsolationChannel.create(
+            () ->
+                remoteChannel(

Review Comment:
   could make a Supplier for channel that you pass to IsolationChannel or call 
directly to reduce duplication.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java:
##########
@@ -132,9 +132,6 @@ public ManagedChannel get(WindmillServiceAddress 
windmillServiceAddress) {
   public synchronized void consumeFlowControlSettings(
       UserWorkerGrpcFlowControlSettings flowControlSettings) {
     if (!flowControlSettings.equals(currentFlowControlSettings)) {
-      // Refreshing the cache will asynchronously terminate the old channels 
via the removalListener
-      // and return a newly created one on the next Cache.load(address). This 
could be expensive so

Review Comment:
   keep comment?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -619,28 +640,35 @@ private static void 
validateWorkerOptions(DataflowWorkerHarnessOptions options)
         StreamingDataflowWorker.class.getSimpleName());
   }
 
-  private static ChannelCache createChannelCache(
-      DataflowWorkerHarnessOptions workerOptions, ComputationConfig.Fetcher 
configFetcher) {
-    ChannelCache channelCache =
-        ChannelCache.create(
-            (currentFlowControlSettings, serviceAddress) -> {
-              // IsolationChannel will create and manage separate RPC channels 
to the same
-              // serviceAddress.
-              return IsolationChannel.create(
+  private static ChannelCache createChannelCacheForLocalWindmillService(
+      DataflowWorkerHarnessOptions workerOptions) {
+    return ChannelCache.create(
+        (currentFlowControlSettings, serviceAddress) -> {
+          // IsolationChannel will create and manage separate RPC channels to 
the same
+          // serviceAddress.
+          if (serviceAddress.getKind()
+              == 
WindmillServiceAddress.Kind.AUTHENTICATED_GCP_SERVICE_ADDRESS) {
+            return IsolationChannel.create(
+                () ->
+                    remoteChannel(
+                        serviceAddress,
+                        
workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(),
+                        currentFlowControlSettings),
+                currentFlowControlSettings.getOnReadyThresholdBytes());
+          }
+          return workerOptions.getUseWindmillIsolatedChannels()
+              ? IsolationChannel.create(

Review Comment:
   this seems the same as above, and the remote channel creation is always the 
same. Can we simplify as
   Supplier<...> createChannel = () -> remoteChannel(....);
   if ((serviceAddress.getKind()
                 == 
WindmillServiceAddress.Kind.AUTHENTICATED_GCP_SERVICE_ADDRESS || 
workerOptions.getUseWindmillIsolatedChannels()) {
     return IsolationChannel.create(createChannel, 
currentFlowControlSettings.getOnReadThresholdBytes());
   }
   return createChannel();
   }



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -520,11 +509,43 @@ public static StreamingDataflowWorker 
fromOptions(DataflowWorkerHarnessOptions o
         workFailureProcessor,
         streamingCounters,
         memoryMonitor,
-        
configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(),
         Executors.newSingleThreadScheduledExecutor(
             new ThreadFactoryBuilder().setNameFormat("RefreshWork").build()),
         stageInfo,
-        
configFetcherComputationStateCacheAndWindmillClient.windmillDispatcherClient());
+        dependencies);
+  }
+
+  /**
+   * Returns dependencies with {@link ChannelCache}, {@link
+   * 
org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactory},
 and
+   * {@link GrpcDispatcherClient} set.

Review Comment:
   can add ConfigFetcher



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