scwhittle commented on code in PR #34539: URL: https://github.com/apache/beam/pull/34539#discussion_r2035586537
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the Review Comment: right away does autosizing allow specifying a minimum? is the initial a minimum? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the + // user worker and Windmill that attempt to guard the process from OOMing (i.e + // MemoryMonitor.java). + .setEnableAutoFlowControl(false) + .build(); + + private static final UserWorkerGrpcFlowControlSettings DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setEnableAutoFlowControl(false) + .build(); + + private WindmillChannels() {} + + public static ManagedChannel inProcessChannel(String channelName) { + return InProcessChannelBuilder.forName(channelName).directExecutor().build(); + } + + public static Channel localhostChannel(int port) { + return NettyChannelBuilder.forAddress(LOCALHOST, port) + .maxInboundMessageSize(Integer.MAX_VALUE) + .negotiationType(NegotiationType.PLAINTEXT) + .build(); + } + + public static ManagedChannel remoteChannel( + WindmillServiceAddress windmillServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + switch (windmillServiceAddress.getKind()) { + case GCP_SERVICE_ADDRESS: + return remoteChannel( + windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); + case AUTHENTICATED_GCP_SERVICE_ADDRESS: + return remoteDirectChannel( + windmillServiceAddress.authenticatedGcpServiceAddress(), + windmillServiceRpcChannelTimeoutSec, + flowControlSettings); + default: + throw new UnsupportedOperationException( + "Only GCP_SERVICE_ADDRESS and AUTHENTICATED_GCP_SERVICE_ADDRESS are supported" + + " WindmillServiceAddresses."); + } + } + + public static UserWorkerGrpcFlowControlSettings getDefaultDirectpathFlowControlSettings() { Review Comment: just make the constants public and remove the method? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the + // user worker and Windmill that attempt to guard the process from OOMing (i.e + // MemoryMonitor.java). + .setEnableAutoFlowControl(false) + .build(); + + private static final UserWorkerGrpcFlowControlSettings DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) Review Comment: remove this from the default until we try it out? ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java: ########## @@ -132,12 +136,80 @@ public void testClear() throws InterruptedException { CountDownLatch notifyWhenChannelClosed = new CountDownLatch(1); cache = ChannelCache.forTesting( - ignored -> newChannel(channelName), notifyWhenChannelClosed::countDown); + (a, b) -> newChannel(channelName), notifyWhenChannelClosed::countDown); WindmillServiceAddress someAddress = mock(WindmillServiceAddress.class); ManagedChannel cachedChannel = cache.get(someAddress); cache.clear(); notifyWhenChannelClosed.await(); assertTrue(cache.isEmpty()); assertTrue(cachedChannel.isShutdown()); } + + @Test + public void testConsumeFlowControlSettings() throws InterruptedException { + String channelName = "channel"; + CountDownLatch notifyWhenChannelClosed = new CountDownLatch(1); + AtomicInteger newChannelsCreated = new AtomicInteger(); + cache = + ChannelCache.forTesting( + (a, b) -> { + ManagedChannel channel = newChannel(channelName); + newChannelsCreated.incrementAndGet(); + return channel; + }, + notifyWhenChannelClosed::countDown); + WindmillServiceAddress someAddress = mock(WindmillServiceAddress.class); + when(someAddress.getKind()) + .thenReturn(WindmillServiceAddress.Kind.AUTHENTICATED_GCP_SERVICE_ADDRESS); + ManagedChannel cachedChannel = cache.get(someAddress); + cache.consumeFlowControlSettings( + UserWorkerGrpcFlowControlSettings.newBuilder() + .setEnableAutoFlowControl(true) + .setOnReadyThresholdBytes(1) + .setFlowControlWindowBytes(1) + .build()); + ManagedChannel reloadedChannel = cache.get(someAddress); + notifyWhenChannelClosed.await(); + assertThat(cachedChannel).isNotSameInstanceAs(reloadedChannel); + assertTrue(cachedChannel.isShutdown()); + assertFalse(reloadedChannel.isShutdown()); + assertThat(newChannelsCreated.get()).isEqualTo(2); + assertThat(cache.get(someAddress)).isSameInstanceAs(reloadedChannel); + } + + @Test + public void testConsumeFlowControlSettings_sameFlowControlSettings() { + String channelName = "channel"; + AtomicInteger newChannelsCreated = new AtomicInteger(); + UserWorkerGrpcFlowControlSettings flowControlSettings = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setEnableAutoFlowControl(true) + .setOnReadyThresholdBytes(1) + .setFlowControlWindowBytes(1) + .build(); + cache = + ChannelCache.forTesting( + (a, b) -> { + ManagedChannel channel = newChannel(channelName); Review Comment: ditto ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) Review Comment: how about a different constant used here and below for onready threshold. Can set them to the same value for now but less brittle to change. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the + // user worker and Windmill that attempt to guard the process from OOMing (i.e + // MemoryMonitor.java). + .setEnableAutoFlowControl(false) + .build(); + + private static final UserWorkerGrpcFlowControlSettings DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setEnableAutoFlowControl(false) + .build(); + + private WindmillChannels() {} + + public static ManagedChannel inProcessChannel(String channelName) { + return InProcessChannelBuilder.forName(channelName).directExecutor().build(); + } + + public static Channel localhostChannel(int port) { + return NettyChannelBuilder.forAddress(LOCALHOST, port) + .maxInboundMessageSize(Integer.MAX_VALUE) + .negotiationType(NegotiationType.PLAINTEXT) + .build(); + } + + public static ManagedChannel remoteChannel( + WindmillServiceAddress windmillServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + switch (windmillServiceAddress.getKind()) { + case GCP_SERVICE_ADDRESS: + return remoteChannel( + windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); Review Comment: seems like the DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS aren't being used if not passed here. Seems like we should pass in and use in remoteChannel when configuring. That would let us play around with the settings for non-direct jobs also. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java: ########## @@ -108,7 +112,7 @@ public void testRemoveAndClose() throws InterruptedException { CountDownLatch notifyWhenChannelClosed = new CountDownLatch(1); cache = ChannelCache.forTesting( - ignored -> newChannel(channelName), notifyWhenChannelClosed::countDown); + (a, b) -> newChannel(channelName), notifyWhenChannelClosed::countDown); Review Comment: replace a,b with ignoredFlowControlSettings, ignoredAddress throughout I'd rather have test be a little wordier than confusing ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java: ########## @@ -76,9 +93,14 @@ public static ChannelCache create( new ThreadFactoryBuilder().setNameFormat("GrpcChannelCloser").build())); } + public static ChannelCache create( Review Comment: maybe better to remove this and update callers to ignore settings? then we can see at that call-site easier that they are present but ignored ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java: ########## @@ -621,18 +623,45 @@ private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) } private static ChannelCachingStubFactory createFanOutStubFactory( - DataflowWorkerHarnessOptions workerOptions) { - return ChannelCachingRemoteStubFactory.create( - workerOptions.getGcpCredential(), + DataflowWorkerHarnessOptions workerOptions, ComputationConfig.Fetcher configFetcher) { + ChannelCache channelCache = ChannelCache.create( - serviceAddress -> - // IsolationChannel will create and manage separate RPC channels to the same - // serviceAddress. - IsolationChannel.create( - () -> - remoteChannel( - serviceAddress, - workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec())))); + (currentFlowControlSettings, serviceAddress) -> { + // IsolationChannel will create and manage separate RPC channels to the same + // serviceAddress. + return IsolationChannel.create( + () -> + remoteChannel( + serviceAddress, + workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(), + currentFlowControlSettings), + currentFlowControlSettings.getOnReadyThresholdBytes()); + }); + channelCache.consumeFlowControlSettings(resolveInitialFlowControlSettings(configFetcher)); Review Comment: I think this line and the resolve method in this file can be removed, the cache internally can resolve unset or default settings appropriately. the fetcher observer should notify if there are overridden settings ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java: ########## @@ -132,12 +136,80 @@ public void testClear() throws InterruptedException { CountDownLatch notifyWhenChannelClosed = new CountDownLatch(1); cache = ChannelCache.forTesting( - ignored -> newChannel(channelName), notifyWhenChannelClosed::countDown); + (a, b) -> newChannel(channelName), notifyWhenChannelClosed::countDown); WindmillServiceAddress someAddress = mock(WindmillServiceAddress.class); ManagedChannel cachedChannel = cache.get(someAddress); cache.clear(); notifyWhenChannelClosed.await(); assertTrue(cache.isEmpty()); assertTrue(cachedChannel.isShutdown()); } + + @Test + public void testConsumeFlowControlSettings() throws InterruptedException { + String channelName = "channel"; + CountDownLatch notifyWhenChannelClosed = new CountDownLatch(1); + AtomicInteger newChannelsCreated = new AtomicInteger(); + cache = + ChannelCache.forTesting( + (a, b) -> { Review Comment: seems like you should keep track of what the settings passed in are, to make sure the plumbing is right. for example could add the settings to some concurrentqueue and then validate below ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the + // user worker and Windmill that attempt to guard the process from OOMing (i.e + // MemoryMonitor.java). + .setEnableAutoFlowControl(false) + .build(); + + private static final UserWorkerGrpcFlowControlSettings DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setEnableAutoFlowControl(false) + .build(); + + private WindmillChannels() {} + + public static ManagedChannel inProcessChannel(String channelName) { + return InProcessChannelBuilder.forName(channelName).directExecutor().build(); + } + + public static Channel localhostChannel(int port) { + return NettyChannelBuilder.forAddress(LOCALHOST, port) + .maxInboundMessageSize(Integer.MAX_VALUE) + .negotiationType(NegotiationType.PLAINTEXT) + .build(); + } + + public static ManagedChannel remoteChannel( + WindmillServiceAddress windmillServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + switch (windmillServiceAddress.getKind()) { + case GCP_SERVICE_ADDRESS: + return remoteChannel( + windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); + case AUTHENTICATED_GCP_SERVICE_ADDRESS: + return remoteDirectChannel( + windmillServiceAddress.authenticatedGcpServiceAddress(), + windmillServiceRpcChannelTimeoutSec, + flowControlSettings); + default: + throw new UnsupportedOperationException( + "Only GCP_SERVICE_ADDRESS and AUTHENTICATED_GCP_SERVICE_ADDRESS are supported" + + " WindmillServiceAddresses."); + } + } + + public static UserWorkerGrpcFlowControlSettings getDefaultDirectpathFlowControlSettings() { + return DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS; + } + + public static UserWorkerGrpcFlowControlSettings getDefaultCloudpathFlowControlSettings() { + return DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS; + } + + private static ManagedChannel remoteDirectChannel( + AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + NettyChannelBuilder channelBuilder = + withDefaultChannelOptions( + NettyChannelBuilder.forAddress( + authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), + // Ports are required for direct channels. + authenticatedGcpServiceAddress.gcpServiceAddress().getPort(), + new AltsChannelCredentials.Builder().build()) + .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()), + windmillServiceRpcChannelTimeoutSec); + int flowControlWindowSizeBytes = + Math.max(WINDMILL_MIN_FLOW_CONTROL_WINDOW, flowControlSettings.getFlowControlWindowBytes()); + return flowControlSettings.getEnableAutoFlowControl() + ? channelBuilder.initialFlowControlWindow(flowControlWindowSizeBytes).build() + : channelBuilder.flowControlWindow(flowControlWindowSizeBytes).build(); + } + + public static ManagedChannel remoteChannel( + HostAndPort endpoint, int windmillServiceRpcChannelTimeoutSec) { + return withDefaultChannelOptions( + NettyChannelBuilder.forAddress( + endpoint.getHost(), + endpoint.hasPort() + ? endpoint.getPort() + : WindmillEndpoints.DEFAULT_WINDMILL_SERVICE_PORT), + windmillServiceRpcChannelTimeoutSec) + .negotiationType(NegotiationType.TLS) + .sslContext(dataflowGrpcSslContext(endpoint)) Review Comment: should we be doing this for remoteDirectChannel? ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java: ########## @@ -45,7 +49,7 @@ public class ChannelCacheTest { private static ChannelCache newCache( Function<WindmillServiceAddress, ManagedChannel> channelFactory) { - return ChannelCache.forTesting(channelFactory, () -> {}); + return ChannelCache.forTesting((ignored, address) -> channelFactory.apply(address), () -> {}); Review Comment: ignoredFlowControlSettings ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillStubFactoryFactoryImpl.java: ########## @@ -39,10 +39,12 @@ public WindmillStubFactoryFactoryImpl(DataflowWorkerHarnessOptions workerOptions @Override public WindmillStubFactory makeWindmillStubFactory(boolean useIsolatedChannels) { Function<WindmillServiceAddress, ManagedChannel> channelFactory = - serviceAddress -> remoteChannel(serviceAddress, windmillServiceRpcChannelAliveTimeoutSec); + serviceAddress -> + remoteChannel( + serviceAddress.getServiceAddress(), windmillServiceRpcChannelAliveTimeoutSec); ChannelCache channelCache = ChannelCache.create( - serviceAddress -> + (ignored, serviceAddress) -> Review Comment: how about ignoredFlowControlSettings (or don't ignore it if we change remoteChannel to use it) ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/WindmillChannels.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerGrpcFlowControlSettings; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.alts.AltsChannelCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.grpc.v1p69p0.io.netty.handler.ssl.SslContext; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utility class used to create different RPC Channels. */ +public final class WindmillChannels { + public static final String LOCALHOST = "localhost"; + private static final int MAX_REMOTE_TRACE_EVENTS = 100; + // 1MiB. + private static final int MAX_INBOUND_METADATA_SIZE_BYTES = 1024 * 1024; + // 10MiB. Roughly 2x max message size. + private static final int WINDMILL_MIN_FLOW_CONTROL_WINDOW = + NettyChannelBuilder.DEFAULT_FLOW_CONTROL_WINDOW * 10; + + // User a bigger flow control window and onready threshold for directpath to prevent churn when + // gRPC is trying to flush gRPCs over the wire. If onReadyThreshold and flowControlWindowBytes are + // too low, it was observed in testing that the gRPC channel can get stuck in a "not-ready" state + // until stream deadline. + private static final UserWorkerGrpcFlowControlSettings DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + // 100MiB. + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // 100MiB. + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW * 10) + // Prevent gRPC from automatically resizing the window. If we have things to send/receive + // from Windmill we want to do it right way. There are internal pushback mechanisms in the + // user worker and Windmill that attempt to guard the process from OOMing (i.e + // MemoryMonitor.java). + .setEnableAutoFlowControl(false) + .build(); + + private static final UserWorkerGrpcFlowControlSettings DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS = + UserWorkerGrpcFlowControlSettings.newBuilder() + .setFlowControlWindowBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setOnReadyThresholdBytes(WINDMILL_MIN_FLOW_CONTROL_WINDOW) + .setEnableAutoFlowControl(false) + .build(); + + private WindmillChannels() {} + + public static ManagedChannel inProcessChannel(String channelName) { + return InProcessChannelBuilder.forName(channelName).directExecutor().build(); + } + + public static Channel localhostChannel(int port) { + return NettyChannelBuilder.forAddress(LOCALHOST, port) + .maxInboundMessageSize(Integer.MAX_VALUE) + .negotiationType(NegotiationType.PLAINTEXT) + .build(); + } + + public static ManagedChannel remoteChannel( + WindmillServiceAddress windmillServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + switch (windmillServiceAddress.getKind()) { + case GCP_SERVICE_ADDRESS: + return remoteChannel( + windmillServiceAddress.gcpServiceAddress(), windmillServiceRpcChannelTimeoutSec); + case AUTHENTICATED_GCP_SERVICE_ADDRESS: + return remoteDirectChannel( + windmillServiceAddress.authenticatedGcpServiceAddress(), + windmillServiceRpcChannelTimeoutSec, + flowControlSettings); + default: + throw new UnsupportedOperationException( + "Only GCP_SERVICE_ADDRESS and AUTHENTICATED_GCP_SERVICE_ADDRESS are supported" + + " WindmillServiceAddresses."); + } + } + + public static UserWorkerGrpcFlowControlSettings getDefaultDirectpathFlowControlSettings() { + return DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS; + } + + public static UserWorkerGrpcFlowControlSettings getDefaultCloudpathFlowControlSettings() { + return DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS; + } + + private static ManagedChannel remoteDirectChannel( + AuthenticatedGcpServiceAddress authenticatedGcpServiceAddress, + int windmillServiceRpcChannelTimeoutSec, + UserWorkerGrpcFlowControlSettings flowControlSettings) { + NettyChannelBuilder channelBuilder = + withDefaultChannelOptions( + NettyChannelBuilder.forAddress( + authenticatedGcpServiceAddress.gcpServiceAddress().getHost(), + // Ports are required for direct channels. + authenticatedGcpServiceAddress.gcpServiceAddress().getPort(), + new AltsChannelCredentials.Builder().build()) + .overrideAuthority(authenticatedGcpServiceAddress.authenticatingService()), + windmillServiceRpcChannelTimeoutSec); + int flowControlWindowSizeBytes = Review Comment: probably have a helper to update based upon flow control settings to share with remote channel. -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org