scwhittle commented on code in PR #30425:
URL: https://github.com/apache/beam/pull/30425#discussion_r1514388493
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java:
##########
@@ -97,6 +97,14 @@ CloudWindmillServiceV1Alpha1Stub getWindmillServiceStub() {
: randomlySelectNextStub(windmillServiceStubs));
}
+ WindmillServiceAddress getWindmillServiceAddress() {
+ ImmutableList<HostAndPort> endpoints =
+ ImmutableList.copyOf(dispatcherStubs.get().dispatcherEndpoints());
Review Comment:
just use getDispatcherEndpoints?
it doesn't seem like we need to copy we just need to make sure we use a
consistent set by keeping the reference to it.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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 java.io.PrintWriter;
+import java.util.Map;
+import java.util.Optional;
+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.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;
+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;
+
+ public ChannelCache(
+ boolean useIsolatedChannels,
+ Function<WindmillServiceAddress, ManagedChannel> channelFactory) {
+ this.channelCache =
+ CacheBuilder.newBuilder()
+ .build(
+ new CacheLoader<WindmillServiceAddress, ManagedChannel>() {
+ @Override
+ public ManagedChannel load(WindmillServiceAddress
serviceAddress) {
+ // IsolationChannel will create and manage separate RPC
channels to the same
+ // serviceAddress via calling the channelFactory, else
just directly return the
+ // RPC channel.
+ return useIsolatedChannels
+ ? IsolationChannel.create(() ->
channelFactory.apply(serviceAddress))
+ : channelFactory.apply(serviceAddress);
+ }
+ });
+ }
+
+ private static void shutdownChannel(ManagedChannel channel) {
+ channel.shutdown();
+ try {
+ channel.awaitTermination(10, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ LOG.error("Couldn't close gRPC channel={}", channel, e);
+ }
+ channel.shutdownNow();
+ }
+
+ public ManagedChannel get(WindmillServiceAddress windmillServiceAddress) {
+ return channelCache.getUnchecked(windmillServiceAddress);
+ }
+
+ public void removeAndClose(WindmillServiceAddress windmillServiceAddress) {
+ Optional.ofNullable(channelCache.getIfPresent(windmillServiceAddress))
+ .ifPresent(ChannelCache::shutdownChannel);
Review Comment:
it would be nice if this didn't block the caller but happened in the
background as it is just to help clean up. Otherwise if we get some metadata
update that triggers removing X windmill worker endpoints we could delay up to
10*X seconds.
It looks like Caffeine is recommended over guava cachebuilder from [javadoc](
https://guava.dev/releases/snapshot/api/docs/com/google/common/cache/CacheBuilder.html)
and it supports a removalListener that will run on an executor which sounds
like it would do this
[javadoc](https://www.javadoc.io/doc/com.github.ben-manes.caffeine/caffeine/latest/com.github.benmanes.caffeine/com/github/benmanes/caffeine/cache/Caffeine.html)
--
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]