scwhittle commented on code in PR #34653:
URL: https://github.com/apache/beam/pull/34653#discussion_r2068354018
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -645,6 +663,36 @@ private static ChannelCache createChannelCache(
return channelCache;
}
+ private static ChannelCache createChannelCache(DataflowWorkerHarnessOptions
workerOptions) {
+ return ChannelCache.create(
+ (currentFlowControlSettings, serviceAddress) -> {
+ // IsolationChannel will create and manage separate RPC channels to
the same
Review Comment:
related to above simplification, can we always just use the
configawarechannelfactory? it seems like this is the same logic
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -522,11 +517,46 @@ public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions o
workFailureProcessor,
streamingCounters,
memoryMonitor,
-
configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(),
+ dependencies.windmillStreamFactory(),
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat("RefreshWork").build()),
stageInfo,
-
configFetcherComputationStateCacheAndWindmillClient.windmillDispatcherClient());
+ dependencies.windmillDispatcherClient(),
+ dependencies.channelCache(),
+ dependencies.stubFactory());
+ }
+
+ private static Dependencies.Builder initialDependencies(
+ DataflowWorkerHarnessOptions options, ComputationConfig.Fetcher
configFetcher) {
+ if (options.getUseWindmillIsolatedChannels() == null
+ || options.getIsWindmillServiceDirectPathEnabled()) {
+ ConfigAwareChannelFactory channelFactory =
+ new
ConfigAwareChannelFactory(options.getWindmillServiceRpcChannelAliveTimeoutSec());
+ ChannelCache channelCache = ChannelCache.create(channelFactory);
+ ChannelCachingRemoteStubFactory stubFactory =
+ ChannelCachingRemoteStubFactory.create(options.getGcpCredential(),
channelCache);
+ GrpcDispatcherClient dispatcherClient =
GrpcDispatcherClient.create(stubFactory);
+ configFetcher
+ .getGlobalConfigHandle()
+ .registerConfigObserver(
+ config -> {
+ if (channelFactory.tryConsumeJobConfig(config)) {
Review Comment:
if I'm reading things correctly we're no longer updating the channel cache
with the flow control setttings in this path anymore.
Should we remove that logic from the channel cache and instead just do it
inside the channelfactory as well? Ie change channel factory method from
taking in the flow control settings as a parameter and just treat that again as
something that changes how channels are constructed and triggers a refresh.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -522,11 +517,46 @@ public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions o
workFailureProcessor,
streamingCounters,
memoryMonitor,
-
configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(),
+ dependencies.windmillStreamFactory(),
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat("RefreshWork").build()),
stageInfo,
-
configFetcherComputationStateCacheAndWindmillClient.windmillDispatcherClient());
+ dependencies.windmillDispatcherClient(),
+ dependencies.channelCache(),
+ dependencies.stubFactory());
+ }
+
+ private static Dependencies.Builder initialDependencies(
+ DataflowWorkerHarnessOptions options, ComputationConfig.Fetcher
configFetcher) {
+ if (options.getUseWindmillIsolatedChannels() == null
+ || options.getIsWindmillServiceDirectPathEnabled()) {
+ ConfigAwareChannelFactory channelFactory =
+ new
ConfigAwareChannelFactory(options.getWindmillServiceRpcChannelAliveTimeoutSec());
+ ChannelCache channelCache = ChannelCache.create(channelFactory);
+ ChannelCachingRemoteStubFactory stubFactory =
+ ChannelCachingRemoteStubFactory.create(options.getGcpCredential(),
channelCache);
+ GrpcDispatcherClient dispatcherClient =
GrpcDispatcherClient.create(stubFactory);
+ configFetcher
+ .getGlobalConfigHandle()
+ .registerConfigObserver(
+ config -> {
+ if (channelFactory.tryConsumeJobConfig(config)) {
+
dispatcherClient.reloadDispatcherEndpoints(config.windmillServiceEndpoints());
+ }
+ });
+ return Dependencies.builder()
+ .setChannelCache(channelCache)
+ .setStubFactory(stubFactory)
+ .setWindmillDispatcherClient(dispatcherClient);
+ } else {
+ ChannelCache channelCache = createConfigAwareChannelCache(options,
configFetcher);
Review Comment:
can we just always set things up to be config aware and push the different
on if it is isolated/directpath enabled down to where we react to changes?
I think that will simplify things in this file where we have a lot of
different ways we wire things up already
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -509,7 +504,7 @@ public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions o
return new StreamingDataflowWorker(
windmillServer,
clientId,
- configFetcherComputationStateCacheAndWindmillClient.configFetcher(),
+ dependencies.configFetcher(),
Review Comment:
should we just pass in dependencies instead of the separate params?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -522,11 +517,46 @@ public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions o
workFailureProcessor,
streamingCounters,
memoryMonitor,
-
configFetcherComputationStateCacheAndWindmillClient.windmillStreamFactory(),
+ dependencies.windmillStreamFactory(),
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat("RefreshWork").build()),
stageInfo,
-
configFetcherComputationStateCacheAndWindmillClient.windmillDispatcherClient());
+ dependencies.windmillDispatcherClient(),
+ dependencies.channelCache(),
+ dependencies.stubFactory());
+ }
+
+ private static Dependencies.Builder initialDependencies(
Review Comment:
initialStreamingEngineDependencies?
also how about a comment
// Returns dependencies with channel cache, stub factory, and dispatcher
client set.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ConfigAwareChannelFactory.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 final class ConfigAwareChannelFactory implements WindmillChannelFactory
{
+ private final int windmillServiceRpcChannelAliveTimeoutSec;
+ @MonotonicNonNull private StreamingGlobalConfig currentConfig = null;
+
+ public ConfigAwareChannelFactory(int
windmillServiceRpcChannelAliveTimeoutSec) {
+ this.windmillServiceRpcChannelAliveTimeoutSec =
windmillServiceRpcChannelAliveTimeoutSec;
+ }
+
+ @Override
+ public synchronized ManagedChannel create(
+ Windmill.UserWorkerGrpcFlowControlSettings flowControlSettings,
+ WindmillServiceAddress serviceAddress) {
+ return currentConfig != null
+ &&
currentConfig.userWorkerJobSettings().getUseWindmillIsolatedChannels()
+ // 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(
+ serviceAddress.getServiceAddress(),
+ windmillServiceRpcChannelAliveTimeoutSec,
+ flowControlSettings))
+ : remoteChannel(
+ serviceAddress.getServiceAddress(),
+ windmillServiceRpcChannelAliveTimeoutSec,
+ flowControlSettings);
+ }
+
+ public synchronized boolean tryConsumeJobConfig(StreamingGlobalConfig
config) {
Review Comment:
// Returns true if the config has changed such that previously created
channels should be recreated.
--
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]