parveensania commented on code in PR #37840: URL: https://github.com/apache/beam/pull/37840#discussion_r2979097306
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/FailoverChannel.java: ########## @@ -0,0 +1,303 @@ +/* + * 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 java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongSupplier; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.CallCredentials; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ConnectivityState; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ForwardingClientCallListener.SimpleForwardingClientCallListener; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Metadata; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.MethodDescriptor; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Status; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link ManagedChannel} that wraps a primary and a fallback channel. + * + * <p>Routes requests to either primary or fallback channel based on two independent failover modes: + * + * <ul> + * <li><b>Connection Status Failover:</b> If the primary channel is not ready for 10+ seconds + * (e.g., during network issues), routes to fallback channel. Switches back as soon as the + * primary channel becomes READY again. + * <li><b>RPC Failover:</b> If primary channel RPC fails with transient errors ({@link + * Status.Code#UNAVAILABLE}, {@link Status.Code#DEADLINE_EXCEEDED}, or {@link + * Status.Code#UNKNOWN}), switches to fallback channel and waits for a 1-hour cooling period + * before retrying primary. + * </ul> + */ +@Internal +public final class FailoverChannel extends ManagedChannel { + private static final Logger LOG = LoggerFactory.getLogger(FailoverChannel.class); + // Time to wait before retrying the primary channel after an RPC-based fallback. + private static final long FALLBACK_COOLING_PERIOD_NANOS = TimeUnit.HOURS.toNanos(1); + private static final long PRIMARY_NOT_READY_WAIT_NANOS = TimeUnit.SECONDS.toNanos(10); + private final ManagedChannel primary; + @Nullable private final ManagedChannel fallback; + @Nullable private final CallCredentials fallbackCallCredentials; + // Set when primary's connection state has been unavailable for too long. + private final AtomicBoolean useFallbackDueToState = new AtomicBoolean(false); + // Set when an RPC on primary fails with a transient error. + private final AtomicBoolean useFallbackDueToRPC = new AtomicBoolean(false); + private final AtomicLong lastRPCFallbackTimeNanos = new AtomicLong(0); + private final AtomicLong primaryNotReadySinceNanos = new AtomicLong(-1); + private final LongSupplier nanoClock; + private final AtomicBoolean stateChangeListenerRegistered = new AtomicBoolean(false); Review Comment: Done. stateChangeListenerRegistered is still outside the State object as it is unrelated to all other Atomics. -- 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]
