-
Notifications
You must be signed in to change notification settings - Fork 34
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
DBZ-4389: Add keepalive pings and handle retriable exceptions #55
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains 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 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
27 changes: 27 additions & 0 deletions
27
src/main/java/io/debezium/connector/vitess/VitessErrorHandler.java
This file contains 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,27 @@ | ||
/* | ||
* Copyright Debezium Authors. | ||
* | ||
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 | ||
*/ | ||
package io.debezium.connector.vitess; | ||
|
||
import static io.grpc.Status.Code.UNAVAILABLE; | ||
|
||
import io.debezium.connector.base.ChangeEventQueue; | ||
import io.debezium.pipeline.ErrorHandler; | ||
import io.grpc.StatusRuntimeException; | ||
|
||
public class VitessErrorHandler extends ErrorHandler { | ||
public VitessErrorHandler(String logicalName, ChangeEventQueue<?> queue) { | ||
super(VitessConnector.class, logicalName, queue); | ||
} | ||
|
||
@Override | ||
protected boolean isRetriable(Throwable throwable) { | ||
if (throwable instanceof StatusRuntimeException) { | ||
final StatusRuntimeException exception = (StatusRuntimeException) throwable; | ||
return exception.getStatus().getCode().equals(UNAVAILABLE); | ||
} | ||
return false; | ||
} | ||
} |
This file contains 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 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 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 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
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.
It is not super clear to me why the channel get closed before this change - for my own understanding, is the channel closed by grpc because it is idle? I think the default idle timeout was 30 min: https://grpc.github.io/grpc-java/javadoc/io/grpc/ManagedChannelBuilder.html#idleTimeout-long-java.util.concurrent.TimeUnit-
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.
After troubleshooting for quite a bit I couldn't make a conclusion. I tried to set idleTimeout to different values and there was no effect at all. I added debug output and found the channel itself is mostly in IDLE state even before the timeout, so this idle setting might be irrelevant. Because there is only one gRPC activity (VStream) and no new gRPCs. I also debugged server-side as well.
The general assumption is that the connection/transport is closed after no active streams for a while. To gRPC this means no pings between peers for a period, so the transport is disconnected. I have searched and found these references:
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 links seem to suggest using idealTimeout over keepAliveTime 🤷
I think it's fine to add this option here, the only concern I have here is it requires corresponding setting on the server side as well (e.g., grpc.keepalive_permit_without_calls)
I'm not sure in this case if it worth adding a TODO for it, I will defer to you and others here
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.
yeah, idealTimeout doesn't work in my verification so concluded that it's not related to idleTimeout, because idleTimeout is about "channel", but our problem is with "transport/connection", which we need to tackle with keepalive pings.
As for keepalive_permit_without_calls, I tested without this option and it works. According to this and this:
I think we do have a "call" or outstanding RPC, that is, the VStream call itself. Therefore without this option it won't affect our keepalive pings. I will need to test it in prod for several passes once I had the snapshots ready. We can add it if we do need this option.