scwhittle commented on code in PR #32503: URL: https://github.com/apache/beam/pull/32503#discussion_r1768210362
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactoryFactoryImpl.java: ########## @@ -0,0 +1,54 @@ +/* + * 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.WindmillChannelFactory.remoteChannel; + +import com.google.auth.Credentials; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; + +public class WindmillStubFactoryFactoryImpl implements WindmillStubFactoryFactory { + + private final int windmillServiceRpcChannelAliveTimeoutSec; + private final Credentials gcpCredential; + + public WindmillStubFactoryFactoryImpl(DataflowWorkerHarnessOptions workerOptions) { + this.gcpCredential = workerOptions.getGcpCredential(); + this.windmillServiceRpcChannelAliveTimeoutSec = + workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(); + } + + @Override + public WindmillStubFactory makeWindmillStubFactory(boolean useIsolatedChannels) { + Function<WindmillServiceAddress, ManagedChannel> channelFactory = + serviceAddress -> remoteChannel(serviceAddress, windmillServiceRpcChannelAliveTimeoutSec); + ChannelCache channelCache = + ChannelCache.create( + 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: would it be simpler to just have IsolatedChannel accept a atomic boolean that it then examines on whether or not it should perform the isolation or not? It seems like we wouldn't have to have as much of the plumbing or the factoryfactory if we just always created an IsolatedChannel and just had a way to disable it's effect. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java: ########## @@ -212,7 +216,16 @@ public <ReqT, RespT> ClientCall<ReqT, RespT> interceptCall( this.client = GrpcWindmillServer.newApplianceTestInstance( - inprocessChannel, new FakeWindmillStubFactory(() -> (ManagedChannel) inprocessChannel)); + inprocessChannel, + new WindmillStubFactoryFactory() { Review Comment: use FakeWindmillStubFactoryFactory? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java: ########## @@ -62,31 +67,58 @@ public class GrpcDispatcherClient { @GuardedBy("this") private final Random rand; + private final WindmillStubFactoryFactory windmillStubFactoryFactory; + + private final AtomicReference<WindmillStubFactory> windmillStubFactory = new AtomicReference<>(); + + private final AtomicBoolean useIsolatedChannels = new AtomicBoolean(); + private final boolean reactToIsolatedChannelsJobSetting; + private GrpcDispatcherClient( - WindmillStubFactory windmillStubFactory, + DataflowWorkerHarnessOptions options, + WindmillStubFactoryFactory windmillStubFactoryFactory, DispatcherStubs initialDispatcherStubs, Random rand) { - this.windmillStubFactory = windmillStubFactory; + this.windmillStubFactoryFactory = windmillStubFactoryFactory; + if (DataflowRunner.hasExperiment( + options, STREAMING_ENGINE_USE_JOB_SETTINGS_FOR_ISOLATED_CHANNELS)) { Review Comment: why do we need experiment in addition to the job setting? -- 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]
