|
87 | 87 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; |
88 | 88 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillServer; |
89 | 89 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; |
| 90 | +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.auth.VendoredCredentialsAdapter; |
90 | 91 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCache; |
91 | 92 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingRemoteStubFactory; |
| 93 | +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.FailoverChannel; |
92 | 94 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.IsolationChannel; |
93 | 95 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactoryFactory; |
94 | 96 | import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactoryFactoryImpl; |
|
113 | 115 | import org.apache.beam.sdk.metrics.MetricsEnvironment; |
114 | 116 | import org.apache.beam.sdk.util.construction.CoderTranslation; |
115 | 117 | import org.apache.beam.sdk.values.WindowedValues; |
| 118 | +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.auth.MoreCallCredentials; |
116 | 119 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; |
117 | 120 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; |
118 | 121 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheStats; |
@@ -382,7 +385,8 @@ private StreamingWorkerHarnessFactoryOutput createFanOutStreamingEngineWorkerHar |
382 | 385 | MemoryMonitor memoryMonitor, |
383 | 386 | GrpcDispatcherClient dispatcherClient) { |
384 | 387 | WeightedSemaphore<Commit> maxCommitByteSemaphore = Commits.maxCommitByteSemaphore(); |
385 | | - ChannelCache channelCache = createChannelCache(options, checkNotNull(configFetcher)); |
| 388 | + ChannelCache channelCache = |
| 389 | + createChannelCache(options, checkNotNull(configFetcher), dispatcherClient); |
386 | 390 | @SuppressWarnings("methodref.receiver.bound") |
387 | 391 | FanOutStreamingEngineWorkerHarness fanOutStreamingEngineWorkerHarness = |
388 | 392 | FanOutStreamingEngineWorkerHarness.create( |
@@ -785,20 +789,32 @@ private static void validateWorkerOptions(DataflowWorkerHarnessOptions options) |
785 | 789 | } |
786 | 790 |
|
787 | 791 | private static ChannelCache createChannelCache( |
788 | | - DataflowWorkerHarnessOptions workerOptions, ComputationConfig.Fetcher configFetcher) { |
| 792 | + DataflowWorkerHarnessOptions workerOptions, |
| 793 | + ComputationConfig.Fetcher configFetcher, |
| 794 | + GrpcDispatcherClient dispatcherClient) { |
789 | 795 | ChannelCache channelCache = |
790 | 796 | ChannelCache.create( |
791 | 797 | (currentFlowControlSettings, serviceAddress) -> { |
792 | | - // IsolationChannel will create and manage separate RPC channels to the same |
793 | | - // serviceAddress. |
| 798 | + // IsolationChannel wrapping FailoverChannel so that each active RPC gets its own |
| 799 | + // FailoverChannel instance. The fallback channel is created lazily, at most once, |
| 800 | + // only if failover is actually needed. |
794 | 801 | return IsolationChannel.create( |
795 | 802 | () -> |
796 | | - remoteChannel( |
797 | | - serviceAddress, |
798 | | - workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(), |
799 | | - currentFlowControlSettings), |
| 803 | + FailoverChannel.create( |
| 804 | + remoteChannel( |
| 805 | + serviceAddress, |
| 806 | + workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(), |
| 807 | + currentFlowControlSettings), |
| 808 | + () -> |
| 809 | + remoteChannel( |
| 810 | + dispatcherClient.getDispatcherEndpoints().iterator().next(), |
| 811 | + workerOptions.getWindmillServiceRpcChannelAliveTimeoutSec(), |
| 812 | + currentFlowControlSettings), |
| 813 | + MoreCallCredentials.from( |
| 814 | + new VendoredCredentialsAdapter(workerOptions.getGcpCredential()))), |
800 | 815 | currentFlowControlSettings.getOnReadyThresholdBytes()); |
801 | 816 | }); |
| 817 | + |
802 | 818 | configFetcher |
803 | 819 | .getGlobalConfigHandle() |
804 | 820 | .registerConfigObserver( |
|
0 commit comments