-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-28372][rpc] Migrate to Akka Artery #22271
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
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -95,14 +95,6 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| "Timeout for all outbound connections. If you should experience problems with connecting to a" | ||
| + " TaskManager due to a slow network, you should increase this value."); | ||
|
|
||
| /** Timeout for the startup of the actor system. */ | ||
| public static final ConfigOption<String> STARTUP_TIMEOUT = | ||
| ConfigOptions.key("akka.startup-timeout") | ||
| .stringType() | ||
| .noDefaultValue() | ||
| .withDescription( | ||
| "Timeout after which the startup of a remote component is considered being failed."); | ||
|
|
||
| /** Override SSL support for the Akka transport. */ | ||
| public static final ConfigOption<Boolean> SSL_ENABLED = | ||
| ConfigOptions.key("akka.ssl.enabled") | ||
|
|
@@ -120,7 +112,7 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .withDescription( | ||
| "Maximum size of messages which are sent between the JobManager and the TaskManagers. If Flink" | ||
| + " fails because messages exceed this limit, then you should increase it. The message size requires a" | ||
| + " size-unit specifier."); | ||
| + " size-unit specifier. Has to be at least 32 KiB"); | ||
|
|
||
| /** Maximum number of messages until another actor is executed by the same thread. */ | ||
| public static final ConfigOption<Integer> DISPATCHER_THROUGHPUT = | ||
|
|
@@ -178,13 +170,15 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .defaultValue(true) | ||
| .withDescription("Exit JVM on fatal Akka errors."); | ||
|
|
||
| /** Milliseconds a gate should be closed for after a remote connection was disconnected. */ | ||
| public static final ConfigOption<Long> RETRY_GATE_CLOSED_FOR = | ||
| ConfigOptions.key("akka.retry-gate-closed-for") | ||
| .longType() | ||
| .defaultValue(50L) | ||
| /** Retry outbound connection only after this backoff. */ | ||
| public static final ConfigOption<String> OUTBOUND_RESTART_BACKOFF = | ||
| ConfigOptions.key("akka.outbound-restart-backoff") | ||
| .stringType() | ||
| .defaultValue("50 ms") | ||
| .withDescription( | ||
| "Milliseconds a gate should be closed for after a remote connection was disconnected."); | ||
| "Retry outbound connection only after this backoff." | ||
| + " Replaces the \"akka.retry-gate-closed-for\" key, which is now deprecated." | ||
| + " For now, if only the deprecated key is set, its value will be picked up."); | ||
|
|
||
| // ================================================== | ||
| // Configurations for fork-join-executor. | ||
|
|
@@ -223,9 +217,39 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .build()); | ||
|
|
||
| // ================================================== | ||
| // Configurations for client-socket-work-pool. | ||
| // Deprecated options | ||
| // ================================================== | ||
|
|
||
| /** | ||
| * Timeout for the startup of the actor system. | ||
| * | ||
| * @deprecated Don't use this option anymore. It has no effect on Flink. | ||
| */ | ||
| @Deprecated | ||
| public static final ConfigOption<String> STARTUP_TIMEOUT = | ||
| ConfigOptions.key("akka.startup-timeout") | ||
| .stringType() | ||
| .noDefaultValue() | ||
| .withDescription( | ||
| "Timeout after which the startup of a remote component is considered being failed."); | ||
|
|
||
| /** | ||
| * Milliseconds a gate should be closed for after a remote connection was disconnected. | ||
| * | ||
| * @deprecated Don't use this option anymore. It has no effect on Flink. | ||
| */ | ||
| @Deprecated | ||
| public static final ConfigOption<Long> RETRY_GATE_CLOSED_FOR = | ||
| ConfigOptions.key("akka.retry-gate-closed-for") | ||
| .longType() | ||
| .defaultValue(50L) | ||
| .withDescription( | ||
| "Milliseconds a gate should be closed for after a remote connection was disconnected." | ||
| + " Replaced by \"akka.outbound-restart-backoff\"." | ||
| + " For now, this is used as a backup if the new option is not set."); | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
| public static final ConfigOption<Integer> CLIENT_SOCKET_WORKER_POOL_SIZE_MIN = | ||
| ConfigOptions.key("akka.client-socket-worker-pool.pool-size-min") | ||
| .intType() | ||
|
|
@@ -235,6 +259,8 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .text("Min number of threads to cap factor-based number to.") | ||
| .build()); | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. needs regeneration of the docs |
||
| public static final ConfigOption<Integer> CLIENT_SOCKET_WORKER_POOL_SIZE_MAX = | ||
| ConfigOptions.key("akka.client-socket-worker-pool.pool-size-max") | ||
| .intType() | ||
|
|
@@ -244,6 +270,8 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .text("Max number of threads to cap factor-based number to.") | ||
| .build()); | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
| public static final ConfigOption<Double> CLIENT_SOCKET_WORKER_POOL_SIZE_FACTOR = | ||
| ConfigOptions.key("akka.client-socket-worker-pool.pool-size-factor") | ||
| .doubleType() | ||
|
|
@@ -257,10 +285,8 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| + " pool-size-max values.") | ||
| .build()); | ||
|
|
||
| // ================================================== | ||
| // Configurations for server-socket-work-pool. | ||
| // ================================================== | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
| public static final ConfigOption<Integer> SERVER_SOCKET_WORKER_POOL_SIZE_MIN = | ||
| ConfigOptions.key("akka.server-socket-worker-pool.pool-size-min") | ||
| .intType() | ||
|
|
@@ -270,6 +296,8 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .text("Min number of threads to cap factor-based number to.") | ||
| .build()); | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
| public static final ConfigOption<Integer> SERVER_SOCKET_WORKER_POOL_SIZE_MAX = | ||
| ConfigOptions.key("akka.server-socket-worker-pool.pool-size-max") | ||
| .intType() | ||
|
|
@@ -279,6 +307,8 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| .text("Max number of threads to cap factor-based number to.") | ||
| .build()); | ||
|
|
||
| /** @deprecated Don't use this option anymore. It has no effect on Flink. */ | ||
| @Deprecated | ||
| public static final ConfigOption<Double> SERVER_SOCKET_WORKER_POOL_SIZE_FACTOR = | ||
| ConfigOptions.key("akka.server-socket-worker-pool.pool-size-factor") | ||
| .doubleType() | ||
|
|
@@ -292,10 +322,6 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con | |
| + " pool-size-max values.") | ||
| .build()); | ||
|
|
||
| // ================================================== | ||
| // Deprecated options | ||
| // ================================================== | ||
|
|
||
| /** | ||
| * The Akka death watch heartbeat interval. | ||
| * | ||
|
|
||
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 |
|---|---|---|
|
|
@@ -96,11 +96,6 @@ under the License. | |
| <artifactId>akka-slf4j_${scala.binary.version}</artifactId> | ||
| <version>${akka.version}</version> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>io.netty</groupId> | ||
| <artifactId>netty</artifactId> | ||
| <version>3.10.6.Final</version> | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. needs a NOTICE update |
||
| </dependency> | ||
|
|
||
| <dependency> | ||
| <groupId>org.slf4j</groupId> | ||
|
|
||
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
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.
Is this a de-facto replacement
RETRY_GATE_CLOSED_FOR?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.
I did this change based on the configuration comments.
classic doc:
artery doc:
The
outbound-restart-backoffcomments are a lot less specific, but they point to the same direction.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.
That was also my conclusion from the docs. Shall we add retry-gate-closed-for as a deprecated key to the new option?
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, that makes sense. I also thought about using the value of
retry-gate-closed-forif that is set explicitly andoutbound-restart-backoffis missing.