-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add a FailoverChannel wrapper on top of IsolationChannel to maintain a set of primary and failover channel. #37840
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
parveensania
wants to merge
4
commits into
apache:master
Choose a base branch
from
parveensania:failover-channel
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+483
−63
Open
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
848f343
Add a wrapper on top of IsolationChannel to maintain a set of primary…
parveensania 234a533
Merge branch 'master' into failover-channel
parveensania 42ec76c
Removing check to verify isolation channel option is set
parveensania b64de5c
Merge branch 'failover-channel' of github.com:parveensania/beam-dp in…
parveensania File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
207 changes: 207 additions & 0 deletions
207
...a/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/FailoverChannel.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,207 @@ | ||
| /* | ||
| * 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.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. It fails over to the | ||
| * fallback channel if the primary channel returns {@link Status#UNAVAILABLE}. | ||
| */ | ||
| @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 a failure, to avoid retrying too quickly | ||
| private static final long FALLBACK_COOLING_PERIOD_NANOS = TimeUnit.HOURS.toNanos(1); | ||
| private final ManagedChannel primary; | ||
| private final ManagedChannel fallback; | ||
| @Nullable private final CallCredentials fallbackCallCredentials; | ||
| private final AtomicBoolean useFallback = new AtomicBoolean(false); | ||
| private final AtomicLong lastFallbackTimeNanos = new AtomicLong(0); | ||
| private final LongSupplier nanoClock; | ||
|
|
||
| private FailoverChannel( | ||
| ManagedChannel primary, | ||
| ManagedChannel fallback, | ||
| @Nullable CallCredentials fallbackCallCredentials, | ||
| LongSupplier nanoClock) { | ||
| this.primary = primary; | ||
| this.fallback = fallback; | ||
| this.fallbackCallCredentials = fallbackCallCredentials; | ||
| this.nanoClock = nanoClock; | ||
| } | ||
|
|
||
| public static FailoverChannel create(ManagedChannel primary, ManagedChannel fallback) { | ||
| return new FailoverChannel(primary, fallback, null, System::nanoTime); | ||
| } | ||
|
|
||
| public static FailoverChannel create( | ||
| ManagedChannel primary, ManagedChannel fallback, CallCredentials fallbackCallCredentials) { | ||
| return new FailoverChannel(primary, fallback, fallbackCallCredentials, System::nanoTime); | ||
| } | ||
|
|
||
| static FailoverChannel forTest( | ||
| ManagedChannel primary, | ||
| ManagedChannel fallback, | ||
| CallCredentials fallbackCallCredentials, | ||
| LongSupplier nanoClock) { | ||
| return new FailoverChannel(primary, fallback, fallbackCallCredentials, nanoClock); | ||
| } | ||
|
|
||
| @Override | ||
| public String authority() { | ||
| return primary.authority(); | ||
| } | ||
|
|
||
| @Override | ||
| public <ReqT, RespT> ClientCall<ReqT, RespT> newCall( | ||
| MethodDescriptor<ReqT, RespT> methodDescriptor, CallOptions callOptions) { | ||
| if (useFallback.get()) { | ||
| long elapsedNanos = nanoClock.getAsLong() - lastFallbackTimeNanos.get(); | ||
| if (elapsedNanos > FALLBACK_COOLING_PERIOD_NANOS) { | ||
| if (useFallback.compareAndSet(true, false)) { | ||
| LOG.info("Fallback cooling period elapsed. Retrying direct path."); | ||
| } | ||
| } else { | ||
| CallOptions fallbackCallOptions = callOptions; | ||
| if (fallbackCallCredentials != null && callOptions.getCredentials() == null) { | ||
| fallbackCallOptions = callOptions.withCallCredentials(fallbackCallCredentials); | ||
| } | ||
| // The boolean `true` marks that the ClientCall is using the | ||
| // fallback (cloudpath) channel. The inner call listener uses this | ||
| // flag so `notifyFailure` will only transition to fallback when a | ||
| // non-fallback (primary) call fails; fallback calls simply log | ||
| // failures and do not re-trigger another fallback transition. | ||
| return new FailoverClientCall<>( | ||
| fallback.newCall(methodDescriptor, fallbackCallOptions), | ||
| true, | ||
| methodDescriptor.getFullMethodName()); | ||
| } | ||
| } | ||
| // The boolean `false` marks that the ClientCall is using the | ||
| // primary (direct) channel. If this call closes with a non-OK status, | ||
| // `notifyFailure` will flip `useFallback` to true, causing subsequent | ||
| // calls to go to the fallback channel for the cooling period. | ||
| return new FailoverClientCall<>( | ||
| primary.newCall(methodDescriptor, callOptions), | ||
| false, | ||
| methodDescriptor.getFullMethodName()); | ||
| } | ||
|
|
||
| @Override | ||
| public ManagedChannel shutdown() { | ||
| primary.shutdown(); | ||
| if (fallback != null) { | ||
| fallback.shutdown(); | ||
| } | ||
| return this; | ||
| } | ||
|
|
||
| @Override | ||
| public ManagedChannel shutdownNow() { | ||
| primary.shutdownNow(); | ||
| if (fallback != null) { | ||
| fallback.shutdownNow(); | ||
| } | ||
| return this; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isShutdown() { | ||
| return primary.isShutdown() && (fallback == null || fallback.isShutdown()); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isTerminated() { | ||
| return primary.isTerminated() && (fallback == null || fallback.isTerminated()); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { | ||
| long endTimeNanos = nanoClock.getAsLong() + unit.toNanos(timeout); | ||
| boolean primaryTerminated = primary.awaitTermination(timeout, unit); | ||
| if (fallback != null) { | ||
| long remainingNanos = Math.max(0, endTimeNanos - nanoClock.getAsLong()); | ||
| return primaryTerminated && fallback.awaitTermination(remainingNanos, TimeUnit.NANOSECONDS); | ||
| } | ||
| return primaryTerminated; | ||
| } | ||
|
|
||
| private void notifyFailure(Status status, boolean isFallback, String methodName) { | ||
| if (!status.isOk() && !isFallback && fallback != null) { | ||
| if (useFallback.compareAndSet(false, true)) { | ||
| lastFallbackTimeNanos.set(nanoClock.getAsLong()); | ||
| LOG.warn( | ||
| "Direct path connection failed with status {} for method: {}. Falling back to" | ||
| + " cloudpath for 1 hour.", | ||
| status, | ||
| methodName); | ||
| } | ||
| } else if (isFallback) { | ||
| LOG.warn("Fallback channel call for method: {} closed with status: {}", methodName, status); | ||
| } | ||
| } | ||
|
|
||
| private final class FailoverClientCall<ReqT, RespT> | ||
| extends SimpleForwardingClientCall<ReqT, RespT> { | ||
| private final boolean isFallback; | ||
| private final String methodName; | ||
|
|
||
| /** | ||
| * @param delegate the underlying ClientCall (either primary or fallback) | ||
| * @param isFallback true if `delegate` is a fallback channel call, false if it is a primary | ||
| * channel call. This flag is inspected by {@link #notifyFailure} to determine whether a | ||
| * failure should trigger switching to the fallback channel (only primary failures do). | ||
| * @param methodName full gRPC method name (for logging) | ||
| */ | ||
| FailoverClientCall(ClientCall<ReqT, RespT> delegate, boolean isFallback, String methodName) { | ||
| super(delegate); | ||
| this.isFallback = isFallback; | ||
| this.methodName = methodName; | ||
| } | ||
|
|
||
| @Override | ||
| public void start(Listener<RespT> responseListener, Metadata headers) { | ||
| super.start( | ||
| new SimpleForwardingClientCallListener<RespT>(responseListener) { | ||
| @Override | ||
| public void onClose(Status status, Metadata trailers) { | ||
| notifyFailure(status, isFallback, methodName); | ||
| super.onClose(status, trailers); | ||
| } | ||
| }, | ||
| headers); | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The javadoc on the class says we fallback only on UNAVAILABLE errors. Based on the code here it looks like we'll fallback on any errors. Is this expected?
https://grpc.io/docs/guides/error/ says network level issues may return UNAVAILABLE or
UNKNOWN or DEADLINE_EXCEEDED. should we include them here?