diff --git a/docs/content.zh/docs/deployment/config.md b/docs/content.zh/docs/deployment/config.md index 4a03c3bc0fdad..7512997bb76ff 100644 --- a/docs/content.zh/docs/deployment/config.md +++ b/docs/content.zh/docs/deployment/config.md @@ -457,10 +457,10 @@ These options are for the network stack that handles the streaming and batch dat {{< generated/all_taskmanager_network_section >}} -### RPC / Akka +### RPC / Pekko -Flink uses Akka for RPC between components (JobManager/TaskManager/ResourceManager). -Flink does not use Akka for data transport. +Flink uses Pekko for RPC between components (JobManager/TaskManager/ResourceManager). +Flink does not use Pekko for data transport. {{< generated/akka_configuration >}} diff --git a/docs/content.zh/docs/deployment/memory/mem_setup_jobmanager.md b/docs/content.zh/docs/deployment/memory/mem_setup_jobmanager.md index 107f04d11ac7d..3cde53d889ca7 100644 --- a/docs/content.zh/docs/deployment/memory/mem_setup_jobmanager.md +++ b/docs/content.zh/docs/deployment/memory/mem_setup_jobmanager.md @@ -95,7 +95,7 @@ Flink 需要多少 *JVM 堆内存*,很大程度上取决于运行的作业数 请参考[常见问题]({{< ref "docs/deployment/memory/mem_trouble" >}}#outofmemoryerror-direct-buffer-memory)。 以下情况可能用到堆外内存: -* Flink 框架依赖(例如 Akka 的网络通信) +* Flink 框架依赖(例如 Pekko 的网络通信) * 在作业提交时(例如一些特殊的批处理 Source)及 Checkpoint 完成的回调函数中执行的用户代码 提示 diff --git a/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md b/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md index f69fee35da44c..a44358b48ed1c 100644 --- a/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md +++ b/docs/content.zh/docs/deployment/resource-providers/standalone/kubernetes.md @@ -309,8 +309,8 @@ data: # 下面几行将公共 libraries 或 connectors 的日志级别保持在 INFO 级别。 # root logger 的配置不会覆盖此处配置。 # 你必须手动修改这里的日志级别。 - logger.akka.name = akka - logger.akka.level = INFO + logger.pekko.name = org.apache.pekko + logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop @@ -380,8 +380,8 @@ data: # 下面几行将公共 libraries 或 connectors 的日志级别保持在 INFO 级别。 # root logger 的配置不会覆盖此处配置。 # 你必须手动修改这里的日志级别。 - logger.akka.name = akka - logger.akka.level = INFO + logger.pekko.name = org.apache.pekko + logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop diff --git a/docs/content.zh/docs/deployment/resource-providers/yarn.md b/docs/content.zh/docs/deployment/resource-providers/yarn.md index 8354af6008a05..474a912c7f9d2 100644 --- a/docs/content.zh/docs/deployment/resource-providers/yarn.md +++ b/docs/content.zh/docs/deployment/resource-providers/yarn.md @@ -211,7 +211,7 @@ Overwriting this configuration parameter can lead to multiple YARN clusters affe - **YARN 2.3.0 < version < 2.4.0**. All containers are restarted if the application master fails. - **YARN 2.4.0 < version < 2.6.0**. TaskManager containers are kept alive across application master failures. This has the advantage that the startup time is faster and that the user does not have to wait for obtaining the container resources again. -- **YARN 2.6.0 <= version**: Sets the attempt failure validity interval to the Flinks' Akka timeout value. The attempt failure validity interval says that an application is only killed after the system has seen the maximum number of application attempts during one interval. This avoids that a long lasting job will deplete it's application attempts. +- **YARN 2.6.0 <= version**: Sets the attempt failure validity interval to the Flinks' Pekko timeout value. The attempt failure validity interval says that an application is only killed after the system has seen the maximum number of application attempts during one interval. This avoids that a long lasting job will deplete it's application attempts. {{< hint danger >}} Hadoop YARN 2.4.0 has a major bug (fixed in 2.5.0) preventing container restarts from a restarted Application Master/Job Manager container. See FLINK-4142 for details. We recommend using at least Hadoop 2.5.0 for high availability setups on YARN.

diff --git a/docs/content.zh/docs/deployment/security/security-ssl.md b/docs/content.zh/docs/deployment/security/security-ssl.md index eae4f9fda925c..78f3a1551526b 100644 --- a/docs/content.zh/docs/deployment/security/security-ssl.md +++ b/docs/content.zh/docs/deployment/security/security-ssl.md @@ -107,7 +107,7 @@ When `security.ssl.internal.enabled` is set to `true`, you can set the following - `taskmanager.data.ssl.enabled`: Data communication between TaskManagers - `blob.service.ssl.enabled`: Transport of BLOBs from JobManager to TaskManager - - `akka.ssl.enabled`: Akka-based RPC connections between JobManager / TaskManager / ResourceManager + - `pekko.ssl.enabled`: Pekko-based RPC connections between JobManager / TaskManager / ResourceManager ### Keystores and Truststores diff --git a/docs/content/docs/deployment/config.md b/docs/content/docs/deployment/config.md index c81e615a6a520..2f9780e140dbf 100644 --- a/docs/content/docs/deployment/config.md +++ b/docs/content/docs/deployment/config.md @@ -459,10 +459,10 @@ These options are for the network stack that handles the streaming and batch dat {{< generated/all_taskmanager_network_section >}} -### RPC / Akka +### RPC / Pekko -Flink uses Akka for RPC between components (JobManager/TaskManager/ResourceManager). -Flink does not use Akka for data transport. +Flink uses Pekko for RPC between components (JobManager/TaskManager/ResourceManager). +Flink does not use Pekko for data transport. {{< generated/akka_configuration >}} diff --git a/docs/content/docs/deployment/memory/mem_setup_jobmanager.md b/docs/content/docs/deployment/memory/mem_setup_jobmanager.md index b77897b2f1b7f..c34e0be6494fc 100644 --- a/docs/content/docs/deployment/memory/mem_setup_jobmanager.md +++ b/docs/content/docs/deployment/memory/mem_setup_jobmanager.md @@ -90,7 +90,7 @@ option. This option can be tuned e.g. if the JobManager process throws ‘OutOfM There can be the following possible sources of *Off-heap* memory consumption: -* Flink framework dependencies (e.g. Akka network communication) +* Flink framework dependencies (e.g. Pekko network communication) * User code executed during job submission (e.g. for certain batch sources) or in checkpoint completion callbacks Note If you have configured the [Total Flink Memory]({{< ref "docs/deployment/memory/mem_setup" >}}#configure-total-memory) diff --git a/docs/content/docs/deployment/resource-providers/standalone/kubernetes.md b/docs/content/docs/deployment/resource-providers/standalone/kubernetes.md index 28bc8f5ee2844..6044530a675a0 100644 --- a/docs/content/docs/deployment/resource-providers/standalone/kubernetes.md +++ b/docs/content/docs/deployment/resource-providers/standalone/kubernetes.md @@ -299,8 +299,8 @@ data: # The following lines keep the log level of common libraries/connectors on # log level INFO. The root logger does not override this. You have to manually # change the log levels here. - logger.akka.name = akka - logger.akka.level = INFO + logger.pekko.name = org.apache.pekko + logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop @@ -369,8 +369,8 @@ data: # The following lines keep the log level of common libraries/connectors on # log level INFO. The root logger does not override this. You have to manually # change the log levels here. - logger.akka.name = akka - logger.akka.level = INFO + logger.pekko.name = org.apache.pekko + logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop diff --git a/docs/content/docs/deployment/resource-providers/yarn.md b/docs/content/docs/deployment/resource-providers/yarn.md index 49d6a3501aaa3..8cfd9b7eb5c64 100644 --- a/docs/content/docs/deployment/resource-providers/yarn.md +++ b/docs/content/docs/deployment/resource-providers/yarn.md @@ -227,7 +227,7 @@ Overwriting this configuration parameter can lead to multiple YARN clusters affe - **YARN 2.3.0 < version < 2.4.0**. All containers are restarted if the application master fails. - **YARN 2.4.0 < version < 2.6.0**. TaskManager containers are kept alive across application master failures. This has the advantage that the startup time is faster and that the user does not have to wait for obtaining the container resources again. -- **YARN 2.6.0 <= version**: Sets the attempt failure validity interval to the Flinks' Akka timeout value. The attempt failure validity interval says that an application is only killed after the system has seen the maximum number of application attempts during one interval. This avoids that a long lasting job will deplete it's application attempts. +- **YARN 2.6.0 <= version**: Sets the attempt failure validity interval to the Flinks' Pekko timeout value. The attempt failure validity interval says that an application is only killed after the system has seen the maximum number of application attempts during one interval. This avoids that a long lasting job will deplete it's application attempts. {{< hint danger >}} Hadoop YARN 2.4.0 has a major bug (fixed in 2.5.0) preventing container restarts from a restarted Application Master/Job Manager container. See FLINK-4142 for details. We recommend using at least Hadoop 2.5.0 for high availability setups on YARN.

diff --git a/docs/content/docs/deployment/security/security-ssl.md b/docs/content/docs/deployment/security/security-ssl.md index 63f376c6adec6..674614b83d895 100644 --- a/docs/content/docs/deployment/security/security-ssl.md +++ b/docs/content/docs/deployment/security/security-ssl.md @@ -107,7 +107,7 @@ When `security.ssl.internal.enabled` is set to `true`, you can set the following - `taskmanager.data.ssl.enabled`: Data communication between TaskManagers - `blob.service.ssl.enabled`: Transport of BLOBs from JobManager to TaskManager - - `akka.ssl.enabled`: Akka-based RPC connections between JobManager / TaskManager / ResourceManager + - `pekko.ssl.enabled`: Pekko-based RPC connections between JobManager / TaskManager / ResourceManager ### Keystores and Truststores diff --git a/docs/content/docs/ops/state/checkpoints.md b/docs/content/docs/ops/state/checkpoints.md index 2d4f6ecd6a893..961db367fc4bd 100644 --- a/docs/content/docs/ops/state/checkpoints.md +++ b/docs/content/docs/ops/state/checkpoints.md @@ -66,7 +66,7 @@ new JobManagerCheckpointStorage(MAX_MEM_STATE_SIZE); Limitations of the `JobManagerCheckpointStorage`: - The size of each individual state is by default limited to 5 MB. This value can be increased in the constructor of the `JobManagerCheckpointStorage`. - - Irrespective of the configured maximal state size, the state cannot be larger than the Akka frame size (see [Configuration]({{< ref "docs/deployment/config" >}})). + - Irrespective of the configured maximal state size, the state cannot be larger than the Pekko frame size (see [Configuration]({{< ref "docs/deployment/config" >}})). - The aggregate state must fit into the JobManager memory. The JobManagerCheckpointStorage is encouraged for: diff --git a/docs/layouts/shortcodes/generated/akka_configuration.html b/docs/layouts/shortcodes/generated/akka_configuration.html index fc64414e16e17..ea3fae1f140e5 100644 --- a/docs/layouts/shortcodes/generated/akka_configuration.html +++ b/docs/layouts/shortcodes/generated/akka_configuration.html @@ -9,121 +9,121 @@ -
akka.ask.callstack
+
pekko.ask.callstack
true Boolean If true, call stack for asynchronous asks are captured. That way, when an ask fails (for example times out), you get a proper exception, describing to the original method call and call site. Note that in case of having millions of concurrent RPC calls, this may add to the memory footprint. -
akka.ask.timeout
+
pekko.ask.timeout
10 s Duration - Timeout used for all futures and blocking Akka calls. If Flink fails due to timeouts then you should try to increase this value. Timeouts can be caused by slow machines or a congested network. The timeout value requires a time-unit specifier (ms/s/min/h/d). + Timeout used for all futures and blocking Pekko calls. If Flink fails due to timeouts then you should try to increase this value. Timeouts can be caused by slow machines or a congested network. The timeout value requires a time-unit specifier (ms/s/min/h/d). -
akka.client-socket-worker-pool.pool-size-factor
+
pekko.client-socket-worker-pool.pool-size-factor
1.0 Double The pool size factor is used to determine thread pool size using the following formula: ceil(available processors * factor). Resulting size is then bounded by the pool-size-min and pool-size-max values. -
akka.client-socket-worker-pool.pool-size-max
+
pekko.client-socket-worker-pool.pool-size-max
2 Integer Max number of threads to cap factor-based number to. -
akka.client-socket-worker-pool.pool-size-min
+
pekko.client-socket-worker-pool.pool-size-min
1 Integer Min number of threads to cap factor-based number to. -
akka.fork-join-executor.parallelism-factor
+
pekko.fork-join-executor.parallelism-factor
2.0 Double The parallelism factor is used to determine thread pool size using the following formula: ceil(available processors * factor). Resulting size is then bounded by the parallelism-min and parallelism-max values. -
akka.fork-join-executor.parallelism-max
+
pekko.fork-join-executor.parallelism-max
64 Integer Max number of threads to cap factor-based parallelism number to. -
akka.fork-join-executor.parallelism-min
+
pekko.fork-join-executor.parallelism-min
8 Integer Min number of threads to cap factor-based parallelism number to. -
akka.framesize
+
pekko.framesize
"10485760b" String 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. -
akka.jvm-exit-on-fatal-error
+
pekko.jvm-exit-on-fatal-error
true Boolean - Exit JVM on fatal Akka errors. + Exit JVM on fatal Pekko errors. -
akka.log.lifecycle.events
+
pekko.log.lifecycle.events
false Boolean - Turns on the Akka’s remote logging of events. Set this value to 'true' in case of debugging. + Turns on the Pekko’s remote logging of events. Set this value to 'true' in case of debugging. -
akka.lookup.timeout
+
pekko.lookup.timeout
10 s Duration Timeout used for the lookup of the JobManager. The timeout value has to contain a time-unit specifier (ms/s/min/h/d). -
akka.retry-gate-closed-for
+
pekko.retry-gate-closed-for
50 Long Milliseconds a gate should be closed for after a remote connection was disconnected. -
akka.server-socket-worker-pool.pool-size-factor
+
pekko.server-socket-worker-pool.pool-size-factor
1.0 Double The pool size factor is used to determine thread pool size using the following formula: ceil(available processors * factor). Resulting size is then bounded by the pool-size-min and pool-size-max values. -
akka.server-socket-worker-pool.pool-size-max
+
pekko.server-socket-worker-pool.pool-size-max
2 Integer Max number of threads to cap factor-based number to. -
akka.server-socket-worker-pool.pool-size-min
+
pekko.server-socket-worker-pool.pool-size-min
1 Integer Min number of threads to cap factor-based number to. -
akka.ssl.enabled
+
pekko.ssl.enabled
true Boolean Turns on SSL for Akka’s remote communication. This is applicable only when the global ssl flag security.ssl.enabled is set to true. -
akka.startup-timeout
+
pekko.startup-timeout
(none) String Timeout after which the startup of a remote component is considered being failed. -
akka.tcp.timeout
+
pekko.tcp.timeout
"20 s" String 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. -
akka.throughput
+
pekko.throughput
15 Integer Number of messages that are processed in a batch before returning the thread to the pool. Low values denote a fair scheduling whereas high values can increase the performance at the cost of unfairness. diff --git a/docs/layouts/shortcodes/generated/all_taskmanager_section.html b/docs/layouts/shortcodes/generated/all_taskmanager_section.html index 29e2a10d57ed0..691475abb3128 100644 --- a/docs/layouts/shortcodes/generated/all_taskmanager_section.html +++ b/docs/layouts/shortcodes/generated/all_taskmanager_section.html @@ -109,7 +109,7 @@
taskmanager.slot.timeout
10 s Duration - Timeout used for identifying inactive slots. The TaskManager will free the slot if it does not become active within the given amount of time. Inactive slots can be caused by an out-dated slot request. If no value is configured, then it will fall back to akka.ask.timeout. + Timeout used for identifying inactive slots. The TaskManager will free the slot if it does not become active within the given amount of time. Inactive slots can be caused by an out-dated slot request. If no value is configured, then it will fall back to pekko.ask.timeout. diff --git a/docs/layouts/shortcodes/generated/metric_configuration.html b/docs/layouts/shortcodes/generated/metric_configuration.html index 841588b57da25..29c571450a121 100644 --- a/docs/layouts/shortcodes/generated/metric_configuration.html +++ b/docs/layouts/shortcodes/generated/metric_configuration.html @@ -24,7 +24,7 @@
metrics.internal.query-service.thread-priority
1 Integer - The thread priority used for Flink's internal metric query service. The thread is created by Akka's thread pool executor. The range of the priority is from 1 (MIN_PRIORITY) to 10 (MAX_PRIORITY). Warning, increasing this value may bring the main Flink components down. + The thread priority used for Flink's internal metric query service. The thread is created by Pekko's thread pool executor. The range of the priority is from 1 (MIN_PRIORITY) to 10 (MAX_PRIORITY). Warning, increasing this value may bring the main Flink components down.
metrics.job.status.enable
diff --git a/docs/layouts/shortcodes/generated/task_manager_configuration.html b/docs/layouts/shortcodes/generated/task_manager_configuration.html index 0bae7d0d794e9..6f2c12d5df3f2 100644 --- a/docs/layouts/shortcodes/generated/task_manager_configuration.html +++ b/docs/layouts/shortcodes/generated/task_manager_configuration.html @@ -127,7 +127,7 @@
taskmanager.slot.timeout
10 s Duration - Timeout used for identifying inactive slots. The TaskManager will free the slot if it does not become active within the given amount of time. Inactive slots can be caused by an out-dated slot request. If no value is configured, then it will fall back to akka.ask.timeout. + Timeout used for identifying inactive slots. The TaskManager will free the slot if it does not become active within the given amount of time. Inactive slots can be caused by an out-dated slot request. If no value is configured, then it will fall back to pekko.ask.timeout. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java index f77498d5d5b36..4eeccccfd590f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/AkkaOptions.java @@ -36,9 +36,10 @@ public class AkkaOptions { @Internal @Documentation.ExcludeFromDocumentation("Internal use only") public static final ConfigOption FORCE_RPC_INVOCATION_SERIALIZATION = - ConfigOptions.key("akka.rpc.force-invocation-serialization") + ConfigOptions.key("pekko.rpc.force-invocation-serialization") .booleanType() .defaultValue(false) + .withDeprecatedKeys("akka.rpc.force-invocation-serialization") .withDescription( Description.builder() .text( @@ -58,22 +59,24 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con /** Flag whether to capture call stacks for RPC ask calls. */ public static final ConfigOption CAPTURE_ASK_CALLSTACK = - ConfigOptions.key("akka.ask.callstack") + ConfigOptions.key("pekko.ask.callstack") .booleanType() .defaultValue(true) + .withDeprecatedKeys("akka.ask.callstack") .withDescription( "If true, call stack for asynchronous asks are captured. That way, when an ask fails " + "(for example times out), you get a proper exception, describing to the original method call and " + "call site. Note that in case of having millions of concurrent RPC calls, this may add to the " + "memory footprint."); - /** Timeout for akka ask calls. */ + /** Timeout for Pekko ask calls. */ public static final ConfigOption ASK_TIMEOUT_DURATION = - ConfigOptions.key("akka.ask.timeout") + ConfigOptions.key("pekko.ask.timeout") .durationType() .defaultValue(Duration.ofSeconds(10)) + .withDeprecatedKeys("akka.ask.timeout") .withDescription( - "Timeout used for all futures and blocking Akka calls. If Flink fails due to timeouts then you" + "Timeout used for all futures and blocking Pekko calls. If Flink fails due to timeouts then you" + " should try to increase this value. Timeouts can be caused by slow machines or a congested network. The" + " timeout value requires a time-unit specifier (ms/s/min/h/d)."); @@ -86,37 +89,41 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con TimeUtils.formatWithHighestUnit(ASK_TIMEOUT_DURATION.defaultValue())) .withDescription(ASK_TIMEOUT_DURATION.description()); - /** The Akka tcp connection timeout. */ + /** The Pekko tcp connection timeout. */ public static final ConfigOption TCP_TIMEOUT = - ConfigOptions.key("akka.tcp.timeout") + ConfigOptions.key("pekko.tcp.timeout") .stringType() .defaultValue("20 s") + .withDeprecatedKeys("akka.tcp.timeout") .withDescription( "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 STARTUP_TIMEOUT = - ConfigOptions.key("akka.startup-timeout") + ConfigOptions.key("pekko.startup-timeout") .stringType() .noDefaultValue() + .withDeprecatedKeys("akka.startup-timeout") .withDescription( "Timeout after which the startup of a remote component is considered being failed."); - /** Override SSL support for the Akka transport. */ + /** Override SSL support for the Pekko transport. */ public static final ConfigOption SSL_ENABLED = - ConfigOptions.key("akka.ssl.enabled") + ConfigOptions.key("pekko.ssl.enabled") .booleanType() .defaultValue(true) + .withDeprecatedKeys("akka.ssl.enabled") .withDescription( "Turns on SSL for Akka’s remote communication. This is applicable only when the global ssl flag" + " security.ssl.enabled is set to true."); - /** Maximum framesize of akka messages. */ + /** Maximum framesize of Pekko messages. */ public static final ConfigOption FRAMESIZE = - ConfigOptions.key("akka.framesize") + ConfigOptions.key("pekko.framesize") .stringType() .defaultValue("10485760b") + .withDeprecatedKeys("akka.framesize") .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" @@ -124,26 +131,29 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con /** Maximum number of messages until another actor is executed by the same thread. */ public static final ConfigOption DISPATCHER_THROUGHPUT = - ConfigOptions.key("akka.throughput") + ConfigOptions.key("pekko.throughput") .intType() .defaultValue(15) + .withDeprecatedKeys("akka.throughput") .withDescription( "Number of messages that are processed in a batch before returning the thread to the pool. Low" + " values denote a fair scheduling whereas high values can increase the performance at the cost of unfairness."); /** Log lifecycle events. */ public static final ConfigOption LOG_LIFECYCLE_EVENTS = - ConfigOptions.key("akka.log.lifecycle.events") + ConfigOptions.key("pekko.log.lifecycle.events") .booleanType() .defaultValue(false) + .withDeprecatedKeys("akka.log.lifecycle.events") .withDescription( - "Turns on the Akka’s remote logging of events. Set this value to 'true' in case of debugging."); + "Turns on the Pekko’s remote logging of events. Set this value to 'true' in case of debugging."); /** Timeout for all blocking calls that look up remote actors. */ public static final ConfigOption LOOKUP_TIMEOUT_DURATION = - ConfigOptions.key("akka.lookup.timeout") + ConfigOptions.key("pekko.lookup.timeout") .durationType() .defaultValue(Duration.ofSeconds(10)) + .withDeprecatedKeys("akka.lookup.timeout") .withDescription( "Timeout used for the lookup of the JobManager. The timeout value has to contain a time-unit" + " specifier (ms/s/min/h/d)."); @@ -171,18 +181,20 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con "DEPRECATED: Use the \"client.timeout\" instead." + " Timeout for all blocking calls on the client side."); - /** Exit JVM on fatal Akka errors. */ + /** Exit JVM on fatal Pekko errors. */ public static final ConfigOption JVM_EXIT_ON_FATAL_ERROR = - ConfigOptions.key("akka.jvm-exit-on-fatal-error") + ConfigOptions.key("pekko.jvm-exit-on-fatal-error") .booleanType() .defaultValue(true) - .withDescription("Exit JVM on fatal Akka errors."); + .withDeprecatedKeys("akka.jvm-exit-on-fatal-error") + .withDescription("Exit JVM on fatal Pekko errors."); /** Milliseconds a gate should be closed for after a remote connection was disconnected. */ public static final ConfigOption RETRY_GATE_CLOSED_FOR = - ConfigOptions.key("akka.retry-gate-closed-for") + ConfigOptions.key("pekko.retry-gate-closed-for") .longType() .defaultValue(50L) + .withDeprecatedKeys("akka.retry-gate-closed-for") .withDescription( "Milliseconds a gate should be closed for after a remote connection was disconnected."); @@ -191,9 +203,10 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con // ================================================== public static final ConfigOption FORK_JOIN_EXECUTOR_PARALLELISM_FACTOR = - ConfigOptions.key("akka.fork-join-executor.parallelism-factor") + ConfigOptions.key("pekko.fork-join-executor.parallelism-factor") .doubleType() .defaultValue(2.0) + .withDeprecatedKeys("akka.fork-join-executor.parallelism-factor") .withDescription( Description.builder() .text( @@ -203,9 +216,10 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con .build()); public static final ConfigOption FORK_JOIN_EXECUTOR_PARALLELISM_MIN = - ConfigOptions.key("akka.fork-join-executor.parallelism-min") + ConfigOptions.key("pekko.fork-join-executor.parallelism-min") .intType() .defaultValue(8) + .withDeprecatedKeys("akka.fork-join-executor.parallelism-min") .withDescription( Description.builder() .text( @@ -213,9 +227,10 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con .build()); public static final ConfigOption FORK_JOIN_EXECUTOR_PARALLELISM_MAX = - ConfigOptions.key("akka.fork-join-executor.parallelism-max") + ConfigOptions.key("pekko.fork-join-executor.parallelism-max") .intType() .defaultValue(64) + .withDeprecatedKeys("akka.fork-join-executor.parallelism-max") .withDescription( Description.builder() .text( @@ -227,27 +242,30 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con // ================================================== public static final ConfigOption CLIENT_SOCKET_WORKER_POOL_SIZE_MIN = - ConfigOptions.key("akka.client-socket-worker-pool.pool-size-min") + ConfigOptions.key("pekko.client-socket-worker-pool.pool-size-min") .intType() .defaultValue(1) + .withDeprecatedKeys("akka.client-socket-worker-pool.pool-size-min") .withDescription( Description.builder() .text("Min number of threads to cap factor-based number to.") .build()); public static final ConfigOption CLIENT_SOCKET_WORKER_POOL_SIZE_MAX = - ConfigOptions.key("akka.client-socket-worker-pool.pool-size-max") + ConfigOptions.key("pekko.client-socket-worker-pool.pool-size-max") .intType() .defaultValue(2) + .withDeprecatedKeys("akka.client-socket-worker-pool.pool-size-max") .withDescription( Description.builder() .text("Max number of threads to cap factor-based number to.") .build()); public static final ConfigOption CLIENT_SOCKET_WORKER_POOL_SIZE_FACTOR = - ConfigOptions.key("akka.client-socket-worker-pool.pool-size-factor") + ConfigOptions.key("pekko.client-socket-worker-pool.pool-size-factor") .doubleType() .defaultValue(1.0) + .withDeprecatedKeys("akka.client-socket-worker-pool.pool-size-factor") .withDescription( Description.builder() .text( @@ -262,27 +280,30 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con // ================================================== public static final ConfigOption SERVER_SOCKET_WORKER_POOL_SIZE_MIN = - ConfigOptions.key("akka.server-socket-worker-pool.pool-size-min") + ConfigOptions.key("pekko.server-socket-worker-pool.pool-size-min") .intType() .defaultValue(1) + .withDeprecatedKeys("akka.server-socket-worker-pool.pool-size-min") .withDescription( Description.builder() .text("Min number of threads to cap factor-based number to.") .build()); public static final ConfigOption SERVER_SOCKET_WORKER_POOL_SIZE_MAX = - ConfigOptions.key("akka.server-socket-worker-pool.pool-size-max") + ConfigOptions.key("pekko.server-socket-worker-pool.pool-size-max") .intType() .defaultValue(2) + .withDeprecatedKeys("akka.server-socket-worker-pool.pool-size-max") .withDescription( Description.builder() .text("Max number of threads to cap factor-based number to.") .build()); public static final ConfigOption SERVER_SOCKET_WORKER_POOL_SIZE_FACTOR = - ConfigOptions.key("akka.server-socket-worker-pool.pool-size-factor") + ConfigOptions.key("pekko.server-socket-worker-pool.pool-size-factor") .doubleType() .defaultValue(1.0) + .withDeprecatedKeys("akka.server-socket-worker-pool.pool-size-factor") .withDescription( Description.builder() .text( @@ -314,7 +335,7 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con + " should decrease this value or increase akka.watch.heartbeat.pause. A thorough description of" + " Akka’s DeathWatch can be found %s", link( - "http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", + "https://pekko.apache.org/docs/pekko/current/remoting-artery.html#failure-detector", "here")) .build()); @@ -337,7 +358,7 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con + " Higher value increases the time to detect a dead TaskManager. A thorough description of Akka’s" + " DeathWatch can be found %s", link( - "http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", + "https://pekko.apache.org/docs/pekko/current/remoting-artery.html#failure-detector", "here")) .build()); @@ -358,7 +379,7 @@ public static boolean isForceRpcInvocationSerializationEnabled(Configuration con + " a high value increases the time to detect a dead TaskManager. A thorough description of Akka’s" + " DeathWatch can be found %s", link( - "http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector", + "https://pekko.apache.org/docs/pekko/current/remoting-artery.html#failure-detector", "here")) .build()); } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java index 65073abf3abb3..79fba8d50a1fc 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java @@ -394,7 +394,7 @@ public static Configuration forReporter(Configuration configuration, String repo .defaultValue(1) .withDescription( "The thread priority used for Flink's internal metric query service. The thread is created" - + " by Akka's thread pool executor. " + + " by Pekko's thread pool executor. " + "The range of the priority is from 1 (MIN_PRIORITY) to 10 (MAX_PRIORITY). " + "Warning, increasing this value may bring the main Flink components down."); /** diff --git a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java index c9f0982e48f7a..716eb89cffca4 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java @@ -273,7 +273,7 @@ public static Configuration forProvider(Configuration configuration, String prov + "Specific internal components (rpc, data transport, blob server) may optionally override " + "this through their own settings."); - /** Enable SSL for internal communication (akka rpc, netty data transport, blob server). */ + /** Enable SSL for internal communication (pekko rpc, netty data transport, blob server). */ @Documentation.Section(Documentation.Sections.SECURITY_SSL) public static final ConfigOption SSL_INTERNAL_ENABLED = key("security.ssl.internal.enabled") diff --git a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java index 8dde8fa70a7de..28cda5e8251ea 100644 --- a/flink-core/src/main/java/org/apache/flink/util/NetUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/NetUtils.java @@ -195,8 +195,8 @@ public static Port getAvailablePort() { // ------------------------------------------------------------------------ /** - * Returns an address in a normalized format for Akka. When an IPv6 address is specified, it - * normalizes the IPv6 address to avoid complications with the exact URL match policy of Akka. + * Returns an address in a normalized format for Pekko. When an IPv6 address is specified, it + * normalizes the IPv6 address to avoid complications with the exact URL match policy of Pekko. * * @param host The hostname, IPv4 or IPv6 address * @return host which will be normalized if it is an IPv6 address @@ -238,9 +238,9 @@ public static String unresolvedHostToNormalizedString(String host) { } /** - * Returns a valid address for Akka. It returns a String of format 'host:port'. When an IPv6 + * Returns a valid address for Pekko. It returns a String of format 'host:port'. When an IPv6 * address is specified, it normalizes the IPv6 address to avoid complications with the exact - * URL match policy of Akka. + * URL match policy of Pekko. * * @param host The hostname, IPv4 or IPv6 address * @param port The port diff --git a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java index 202f4754b1e7c..3c85b9d2c617a 100644 --- a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java +++ b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java @@ -33,7 +33,7 @@ public class TestingUtils { public static final Duration TESTING_DURATION = Duration.ofMinutes(2L); public static final Time TIMEOUT = Time.minutes(1L); - public static final Duration DEFAULT_AKKA_ASK_TIMEOUT = Duration.ofSeconds(200); + public static final Duration DEFAULT_ASK_TIMEOUT = Duration.ofSeconds(200); public static Time infiniteTime() { return Time.milliseconds(Integer.MAX_VALUE); diff --git a/flink-dist/src/main/flink-bin/conf/log4j-console.properties b/flink-dist/src/main/flink-bin/conf/log4j-console.properties index 180e3ab6adc27..0b431501ffec4 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-console.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-console.properties @@ -31,8 +31,8 @@ rootLogger.appenderRef.rolling.ref = RollingFileAppender # The following lines keep the log level of common libraries/connectors on # log level INFO. The root logger does not override this. You have to manually # change the log levels here. -logger.akka.name = akka -logger.akka.level = INFO +logger.pekko.name = org.apache.pekko +logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop diff --git a/flink-dist/src/main/flink-bin/conf/log4j.properties b/flink-dist/src/main/flink-bin/conf/log4j.properties index 477b00b639242..8c0fb7598d6d7 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j.properties @@ -30,8 +30,8 @@ rootLogger.appenderRef.file.ref = MainAppender # The following lines keep the log level of common libraries/connectors on # log level INFO. The root logger does not override this. You have to manually # change the log levels here. -logger.akka.name = akka -logger.akka.level = INFO +logger.pekko.name = org.apache.pekko +logger.pekko.level = INFO logger.kafka.name= org.apache.kafka logger.kafka.level = INFO logger.hadoop.name = org.apache.hadoop diff --git a/flink-dist/src/main/flink-bin/conf/logback-console.xml b/flink-dist/src/main/flink-bin/conf/logback-console.xml index 0c7229d929ddc..28989ddf24df8 100644 --- a/flink-dist/src/main/flink-bin/conf/logback-console.xml +++ b/flink-dist/src/main/flink-bin/conf/logback-console.xml @@ -57,7 +57,7 @@ - + diff --git a/flink-dist/src/main/flink-bin/conf/logback.xml b/flink-dist/src/main/flink-bin/conf/logback.xml index f3c433105a7f9..4bc50acd4cf25 100644 --- a/flink-dist/src/main/flink-bin/conf/logback.xml +++ b/flink-dist/src/main/flink-bin/conf/logback.xml @@ -38,7 +38,7 @@ - + diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index f31d2e2d77eb1..1ae62577a864f 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -389,7 +389,7 @@ function internal_check_logs_for_errors { "Cannot connect to ResourceManager right now" \ "AskTimeoutException" \ "Error while loading kafka-version.properties" \ - "WARN akka.remote.transport.netty.NettyTransport" \ + "WARN org.apache.pekko.remote.transport.netty.NettyTransport" \ "WARN org.jboss.netty.channel.DefaultChannelPipeline" \ "jvm-exit-on-fatal-error" \ 'INFO.*AWSErrorCode' \ @@ -403,7 +403,7 @@ function internal_check_logs_for_errors { "HeapDumpOnOutOfMemoryError" \ "error_prone_annotations" \ "Error sending fetch request" \ - "WARN akka.remote.ReliableDeliverySupervisor" \ + "WARN org.apache.pekko.remote.ReliableDeliverySupervisor" \ "Options.*error_*" \ "not packaged with this application") @@ -449,7 +449,7 @@ function internal_check_logs_for_exceptions { "DisconnectException" \ "Cannot connect to ResourceManager right now" \ "AskTimeoutException" \ - "WARN akka.remote.transport.netty.NettyTransport" \ + "WARN org.apache.pekko.remote.transport.netty.NettyTransport" \ "WARN org.jboss.netty.channel.DefaultChannelPipeline" \ 'INFO.*AWSErrorCode' \ "RejectedExecutionException" \ @@ -465,7 +465,7 @@ function internal_check_logs_for_exceptions { "java.lang.Exception: Artificial failure" \ "org.apache.flink.runtime.checkpoint.CheckpointException" \ "org.apache.flink.runtime.JobException: Recovery is suppressed" \ - "WARN akka.remote.ReliableDeliverySupervisor" \ + "WARN org.apache.pekko.remote.ReliableDeliverySupervisor" \ "RecipientUnreachableException" \ "completeExceptionally" \ "SerializedCheckpointException.unwrap") diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh index 4285214145b1f..7358f5d5e13a3 100755 --- a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -25,7 +25,7 @@ TEST=flink-heavy-deployment-stress-test TEST_PROGRAM_NAME=HeavyDeploymentStressTestProgram TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar -set_config_key "akka.ask.timeout" "60 s" +set_config_key "pekko.ask.timeout" "60 s" set_config_key "web.timeout" "60000" set_config_key "taskmanager.memory.process.size" "1024m" # 1024Mb x 5TMs = 5Gb total heap diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index 9ed24215d35b0..5b32af98761a8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -1342,7 +1342,7 @@ public static ExecutionEnvironment createRemoteEnvironment( /** * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program to a * cluster for execution. Note that all file paths used in the program must be accessible from - * the cluster. The custom configuration file is used to configure Akka specific configuration + * the cluster. The custom configuration file is used to configure Pekko specific configuration * parameters for the Client only; Program parallelism can be set via {@link * ExecutionEnvironment#setParallelism(int)}. * diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java index f92429276075e..ec563ffd96fe2 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java @@ -69,7 +69,7 @@ void testDynamicProperties() throws Exception { new String[] { "-e", KubernetesSessionClusterExecutor.NAME, - "-Dakka.ask.timeout=5 min", + "-Dpekko.ask.timeout=5 min", "-Denv.java.opts=-DappName=foobar" }; @@ -82,7 +82,7 @@ void testDynamicProperties() throws Exception { assertThat(executorConfigMap).hasSize(4); assertThat(executorConfigMap) .contains( - entry("akka.ask.timeout", "5 min"), + entry("pekko.ask.timeout", "5 min"), entry("env.java.opts", "-DappName=foobar")); assertThat(executorConfig.get(DeploymentOptionsInternal.CONF_DIR)) .isEqualTo(confDirPath.toAbsolutePath().toString()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java index f2ed47c419365..966d543bb9768 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java @@ -52,7 +52,7 @@ class KubernetesHighAvailabilityTestBase { public static final String LOCK_IDENTITY = UUID.randomUUID().toString(); public static final String LEADER_ADDRESS = - "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; + "pekko.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; public static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster"; protected static final long TIMEOUT = 30L * 1000L; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java index ccf8fc842a7c7..acbba19fb70ab 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java @@ -56,7 +56,7 @@ class KubernetesLeaderElectionAndRetrievalITCase { private static final String LEADER_CONFIGMAP_NAME = "leader-test-cluster"; private static final String LEADER_ADDRESS = - "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; + "pekko.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; @RegisterExtension static final KubernetesExtension KUBERNETES_EXTENSION = new KubernetesExtension(); diff --git a/flink-python/pyflink/datastream/tests/test_data_stream.py b/flink-python/pyflink/datastream/tests/test_data_stream.py index fc08964976fec..8d8de717639cb 100644 --- a/flink-python/pyflink/datastream/tests/test_data_stream.py +++ b/flink-python/pyflink/datastream/tests/test_data_stream.py @@ -51,7 +51,7 @@ class DataStreamTests(object): def setUp(self) -> None: super(DataStreamTests, self).setUp() config = get_j_env_configuration(self.env._j_stream_execution_environment) - config.setString("akka.ask.timeout", "20 s") + config.setString("pekko.ask.timeout", "20 s") self.test_sink = DataStreamTestSinkFunction() def tearDown(self) -> None: @@ -1319,7 +1319,7 @@ def setUp(self) -> None: self.env.set_parallelism(2) self.env.set_runtime_mode(RuntimeExecutionMode.STREAMING) config = get_j_env_configuration(self.env._j_stream_execution_environment) - config.setString("akka.ask.timeout", "20 s") + config.setString("pekko.ask.timeout", "20 s") self.test_sink = DataStreamTestSinkFunction() def tearDown(self) -> None: diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index de3c4187724d1..e67a16b6cbd15 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -334,7 +334,7 @@ def setUp(self) -> None: super(DataStreamConversionTestCases, self).setUp() config = Configuration() - config.set_string("akka.ask.timeout", "20 s") + config.set_string("pekko.ask.timeout", "20 s") self.env = StreamExecutionEnvironment.get_execution_environment(config) self.t_env = StreamTableEnvironment.create(self.env) diff --git a/flink-rpc/flink-rpc-akka-loader/pom.xml b/flink-rpc/flink-rpc-akka-loader/pom.xml index 61a6966a8ea01..8cc0d2a78c120 100644 --- a/flink-rpc/flink-rpc-akka-loader/pom.xml +++ b/flink-rpc/flink-rpc-akka-loader/pom.xml @@ -55,7 +55,7 @@ under the License. true - + * * diff --git a/flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoader.java b/flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoader.java similarity index 83% rename from flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoader.java rename to flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoader.java index 8796beed58dce..bd38354a51434 100644 --- a/flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoader.java +++ b/flink-rpc/flink-rpc-akka-loader/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; @@ -37,21 +37,23 @@ import java.util.UUID; /** - * Loader for the {@link AkkaRpcSystemLoader}. + * Loader for the {@link PekkoRpcSystemLoader}. * - *

This loader expects the flink-rpc-akka jar to be accessible via {@link + *

This loader expects the flink-rpc-pekko jar to be accessible via {@link * ClassLoader#getResource(String)}. It will extract the jar into a temporary directory and create a * new {@link SubmoduleClassLoader} to load the rpc system from that jar. */ -public class AkkaRpcSystemLoader implements RpcSystemLoader { +public class PekkoRpcSystemLoader implements RpcSystemLoader { static final int LOAD_PRIORITY = 0; - /** The name of the akka dependency jar, bundled with flink-rpc-akka-loader module artifact. */ - private static final String FLINK_RPC_AKKA_FAT_JAR = "flink-rpc-akka.jar"; + /** + * The name of the pekko dependency jar, bundled with flink-rpc-pekko-loader module artifact. + */ + private static final String FLINK_RPC_PEKKO_FAT_JAR = "flink-rpc-akka.jar"; static final String HINT_USAGE = - "mvn clean package -pl flink-rpc/flink-rpc-akka,flink-rpc/flink-rpc-akka-loader -DskipTests"; + "mvn clean package -pl flink-rpc/flink-rpc-pekko,flink-rpc/flink-rpc-akka-loader -DskipTests"; @Override public int getLoadPriority() { @@ -67,14 +69,14 @@ public RpcSystem loadRpcSystem(Configuration config) { Files.createDirectories(FileUtils.getTargetPathIfContainsSymbolicPath(tmpDirectory)); final Path tempFile = Files.createFile( - tmpDirectory.resolve("flink-rpc-akka_" + UUID.randomUUID() + ".jar")); + tmpDirectory.resolve("flink-rpc-akka" + UUID.randomUUID() + ".jar")); final InputStream resourceStream = - flinkClassLoader.getResourceAsStream(FLINK_RPC_AKKA_FAT_JAR); + flinkClassLoader.getResourceAsStream(FLINK_RPC_PEKKO_FAT_JAR); if (resourceStream == null) { throw new RpcLoaderException( String.format( - "Akka RPC system could not be found. If this happened while running a test in the IDE, " + "Pekko RPC system could not be found. If this happened while running a test in the IDE, " + "run '%s' on the command-line, " + "or add a test dependency on the flink-rpc-akka-loader test-jar.", HINT_USAGE)); diff --git a/flink-rpc/flink-rpc-akka-loader/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader b/flink-rpc/flink-rpc-akka-loader/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader index 8387fa4c69658..a2736d3f3ec50 100644 --- a/flink-rpc/flink-rpc-akka-loader/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader +++ b/flink-rpc/flink-rpc-akka-loader/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.runtime.rpc.akka.AkkaRpcSystemLoader +org.apache.flink.runtime.rpc.pekko.PekkoRpcSystemLoader diff --git a/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/FallbackAkkaRpcSystemLoader.java b/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/FallbackPekkoRpcSystemLoader.java similarity index 76% rename from flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/FallbackAkkaRpcSystemLoader.java rename to flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/FallbackPekkoRpcSystemLoader.java index cdafd1ccd74f6..c972e3880acdc 100644 --- a/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/FallbackAkkaRpcSystemLoader.java +++ b/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/FallbackPekkoRpcSystemLoader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.classloading.SubmoduleClassLoader; @@ -42,44 +42,44 @@ /** * Fallback {@link RpcSystemLoader} that does not rely on the flink-rpc-akka fat jar (like {@link - * AkkaRpcSystemLoader}) but instead uses the flink-rpc-akka/target/classes and maven to load the + * PekkoRpcSystemLoader}) but instead uses the flink-rpc-akka/target/classes and maven to load the * rpc system. */ -public class FallbackAkkaRpcSystemLoader implements RpcSystemLoader { - private static final Logger LOG = LoggerFactory.getLogger(FallbackAkkaRpcSystemLoader.class); +public class FallbackPekkoRpcSystemLoader implements RpcSystemLoader { + private static final Logger LOG = LoggerFactory.getLogger(FallbackPekkoRpcSystemLoader.class); private static final String MODULE_FLINK_RPC = "flink-rpc"; - private static final String MODULE_FLINK_RPC_AKKA = "flink-rpc-akka"; + private static final String MODULE_FLINK_RPC_PEKKO = "flink-rpc-akka"; @Override public int getLoadPriority() { - return AkkaRpcSystemLoader.LOAD_PRIORITY + 1; + return PekkoRpcSystemLoader.LOAD_PRIORITY + 1; } @Override public RpcSystem loadRpcSystem(Configuration config) { try { LOG.debug( - "Using Fallback AkkaRpcSystemLoader; this loader will invoke maven to retrieve the dependencies of flink-rpc-akka."); + "Using Fallback PekkoRpcSystemLoader; this loader will invoke maven to retrieve the dependencies of flink-rpc-akka."); final ClassLoader flinkClassLoader = RpcSystem.class.getClassLoader(); // flink-rpc/flink-rpc-akka - final Path akkaRpcModuleDirectory = - findAkkaRpcModuleDirectory(getCurrentWorkingDirectory()); + final Path pekkoRpcModuleDirectory = + findPekkoRpcModuleDirectory(getCurrentWorkingDirectory()); // flink-rpc/flink-rpc-akka/target/classes - final Path akkaRpcModuleClassesDirectory = - akkaRpcModuleDirectory.resolve(Paths.get("target", "classes")); + final Path pekkoRpcModuleClassesDirectory = + pekkoRpcModuleDirectory.resolve(Paths.get("target", "classes")); // flink-rpc/flink-rpc-akka/target/dependencies - final Path akkaRpcModuleDependenciesDirectory = - akkaRpcModuleDirectory.resolve(Paths.get("target", "dependencies")); + final Path pekkoRpcModuleDependenciesDirectory = + pekkoRpcModuleDirectory.resolve(Paths.get("target", "dependencies")); - if (!Files.exists(akkaRpcModuleDependenciesDirectory)) { + if (!Files.exists(pekkoRpcModuleDependenciesDirectory)) { int exitCode = downloadDependencies( - akkaRpcModuleDirectory, akkaRpcModuleDependenciesDirectory); + pekkoRpcModuleDirectory, pekkoRpcModuleDependenciesDirectory); if (exitCode != 0) { throw new RpcLoaderException( "Could not download dependencies of flink-rpc-akka, please see the log output for details."); @@ -87,13 +87,13 @@ public RpcSystem loadRpcSystem(Configuration config) { } else { LOG.debug( "Re-using previously downloaded flink-rpc-akka dependencies. If you are experiencing strange issues, try clearing '{}'.", - akkaRpcModuleDependenciesDirectory); + pekkoRpcModuleDependenciesDirectory); } // assemble URL collection containing target/classes and each jar final List urls = new ArrayList<>(); - urls.add(akkaRpcModuleClassesDirectory.toUri().toURL()); - try (final Stream files = Files.list(akkaRpcModuleDependenciesDirectory)) { + urls.add(pekkoRpcModuleClassesDirectory.toUri().toURL()); + try (final Stream files = Files.list(pekkoRpcModuleDependenciesDirectory)) { final List collect = files.filter(path -> path.getFileName().toString().endsWith(".jar")) .collect(Collectors.toList()); @@ -114,7 +114,7 @@ public RpcSystem loadRpcSystem(Configuration config) { throw new RpcLoaderException( String.format( "Could not initialize RPC system. Run '%s' on the command-line instead.", - AkkaRpcSystemLoader.HINT_USAGE), + PekkoRpcSystemLoader.HINT_USAGE), e); } } @@ -123,7 +123,8 @@ private static Path getCurrentWorkingDirectory() { return Paths.get("").toAbsolutePath(); } - private static Path findAkkaRpcModuleDirectory(Path currentParentCandidate) throws IOException { + private static Path findPekkoRpcModuleDirectory(Path currentParentCandidate) + throws IOException { try (Stream directoryContents = Files.list(currentParentCandidate)) { final Optional flinkRpcModuleDirectory = directoryContents @@ -131,11 +132,11 @@ private static Path findAkkaRpcModuleDirectory(Path currentParentCandidate) thro .findFirst(); if (flinkRpcModuleDirectory.isPresent()) { return flinkRpcModuleDirectory - .map(path -> path.resolve(Paths.get(MODULE_FLINK_RPC_AKKA))) + .map(path -> path.resolve(Paths.get(MODULE_FLINK_RPC_PEKKO))) .get(); } } - return findAkkaRpcModuleDirectory(currentParentCandidate.getParent()); + return findPekkoRpcModuleDirectory(currentParentCandidate.getParent()); } private static int downloadDependencies(Path workingDirectory, Path targetDirectory) diff --git a/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoaderITCase.java b/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoaderITCase.java similarity index 94% rename from flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoaderITCase.java rename to flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoaderITCase.java index 2ba3c621c2be7..61d482229dfdc 100644 --- a/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystemLoaderITCase.java +++ b/flink-rpc/flink-rpc-akka-loader/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystemLoaderITCase.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; @@ -33,14 +33,14 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** - * Tests for the {@link AkkaRpcSystemLoader}. + * Tests for the {@link PekkoRpcSystemLoader}. * *

This must be an ITCase so that it runs after the 'package' phase of maven. Otherwise the * flink-rpc-akka jar will not be available. */ -class AkkaRpcSystemLoaderITCase { +class PekkoRpcSystemLoaderITCase { - private static final AkkaRpcSystemLoader LOADER = new AkkaRpcSystemLoader(); + private static final PekkoRpcSystemLoader LOADER = new PekkoRpcSystemLoader(); @Test void testServiceLoadingWithDefaultConfig() { diff --git a/flink-rpc/flink-rpc-akka-loader/src/test/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader b/flink-rpc/flink-rpc-akka-loader/src/test/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader index eeba46c3e7ea5..0b412dbe7c4ab 100644 --- a/flink-rpc/flink-rpc-akka-loader/src/test/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader +++ b/flink-rpc/flink-rpc-akka-loader/src/test/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystemLoader @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.runtime.rpc.akka.FallbackAkkaRpcSystemLoader +org.apache.flink.runtime.rpc.pekko.FallbackPekkoRpcSystemLoader diff --git a/flink-rpc/flink-rpc-akka/pom.xml b/flink-rpc/flink-rpc-akka/pom.xml index cf1eca38b2743..433c148e623cc 100644 --- a/flink-rpc/flink-rpc-akka/pom.xml +++ b/flink-rpc/flink-rpc-akka/pom.xml @@ -31,9 +31,13 @@ under the License. flink-rpc-akka Flink : RPC : Akka jar + + Pekko-based RPC implementation. Pekko is the Apache fork of Akka. + For compatibility/git reasons not all mentions of Akka have been replaced. + - 2.6.20 + 1.0.0 2.12 2.12.16 @@ -59,15 +63,15 @@ under the License. - com.typesafe.akka - akka-actor_${scala.binary.version} - ${akka.version} + org.apache.pekko + pekko-actor_${scala.binary.version} + ${pekko.version} ${flink.markBundledAsOptional} - com.typesafe.akka - akka-remote_${scala.binary.version} - ${akka.version} + org.apache.pekko + pekko-remote_${scala.binary.version} + ${pekko.version} ${flink.markBundledAsOptional} @@ -83,9 +87,9 @@ under the License. - com.typesafe.akka - akka-slf4j_${scala.binary.version} - ${akka.version} + org.apache.pekko + pekko-slf4j_${scala.binary.version} + ${pekko.version} ${flink.markBundledAsOptional} diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/ActorSystemScheduledExecutorAdapter.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ActorSystemScheduledExecutorAdapter.java similarity index 98% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/ActorSystemScheduledExecutorAdapter.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ActorSystemScheduledExecutorAdapter.java index ff6be5c606fd9..0b6ef6d378c9c 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/ActorSystemScheduledExecutorAdapter.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ActorSystemScheduledExecutorAdapter.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.concurrent.akka; +package org.apache.flink.runtime.concurrent.pekko; import org.apache.flink.runtime.concurrent.ClassLoadingUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.ScheduledExecutor; -import akka.actor.ActorSystem; -import akka.actor.Cancellable; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Cancellable; import javax.annotation.Nonnull; diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/AkkaFutureUtils.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ScalaFutureUtils.java similarity index 95% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/AkkaFutureUtils.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ScalaFutureUtils.java index 5dd96cc80979b..797b60f7969e5 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/akka/AkkaFutureUtils.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/concurrent/pekko/ScalaFutureUtils.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.flink.runtime.concurrent.akka; +package org.apache.flink.runtime.concurrent.pekko; -import akka.dispatch.OnComplete; +import org.apache.pekko.dispatch.OnComplete; import java.util.concurrent.CompletableFuture; @@ -25,7 +25,7 @@ import scala.concurrent.Future; /** Utilities to convert Scala types into Java types. */ -public class AkkaFutureUtils { +public class ScalaFutureUtils { /** * Converts a Scala {@link Future} to a {@link CompletableFuture}. * diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaHandshakeException.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaHandshakeException.java deleted file mode 100644 index 99079e6152022..0000000000000 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaHandshakeException.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.flink.runtime.rpc.akka.exceptions; - -import org.apache.flink.runtime.rpc.exceptions.HandshakeException; - -/** Exception which is thrown if the handshake fails. */ -public class AkkaHandshakeException extends HandshakeException { - private static final long serialVersionUID = 7690464691855200936L; - - public AkkaHandshakeException(String message) { - super(message); - } - - public AkkaHandshakeException(String message, Throwable cause) { - super(message, cause); - } - - public AkkaHandshakeException(Throwable cause) { - super(cause); - } -} diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java deleted file mode 100644 index a37769cdac1ad..0000000000000 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.flink.runtime.rpc.akka.exceptions; - -import org.apache.flink.runtime.rpc.exceptions.RpcException; - -/** Base class for Akka RPC related exceptions. */ -public class AkkaRpcException extends RpcException { - - private static final long serialVersionUID = -3796329968494146418L; - - public AkkaRpcException(String message) { - super(message); - } - - public AkkaRpcException(String message, Throwable cause) { - super(message, cause); - } - - public AkkaRpcException(Throwable cause) { - super(cause); - } -} diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java deleted file mode 100644 index f7967d3b0908d..0000000000000 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcRuntimeException.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.flink.runtime.rpc.akka.exceptions; - -import org.apache.flink.runtime.rpc.exceptions.RpcRuntimeException; - -/** Base class for Akka RPC related runtime exceptions. */ -public class AkkaRpcRuntimeException extends RpcRuntimeException { - public AkkaRpcRuntimeException(String message) { - super(message); - } - - public AkkaRpcRuntimeException(Throwable cause) { - super(cause); - } - - public AkkaRpcRuntimeException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapTools.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java similarity index 85% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapTools.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java index 696821e6a62d7..e0513485f351f 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapTools.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapTools.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; @@ -25,8 +25,8 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelException; -import akka.actor.ActorSystem; import com.typesafe.config.Config; +import org.apache.pekko.actor.ActorSystem; import org.slf4j.Logger; import java.io.IOException; @@ -35,7 +35,7 @@ import java.util.Optional; /** Tools for starting the Actor Systems used to run the JobManager and TaskManager actors. */ -public class AkkaBootstrapTools { +public class ActorSystemBootstrapTools { /** * Starts a remote ActorSystem at given address and specific port range. * @@ -55,13 +55,13 @@ public static ActorSystem startRemoteActorSystem( throws Exception { return startRemoteActorSystem( configuration, - AkkaUtils.getFlinkActorSystemName(), + PekkoUtils.getFlinkActorSystemName(), externalAddress, externalPortRange, NetUtils.getWildcardIPAddress(), Optional.empty(), logger, - AkkaUtils.getForkJoinExecutorConfig( + PekkoUtils.getForkJoinExecutorConfig( getForkJoinExecutorConfiguration(configuration)), null); } @@ -79,7 +79,7 @@ public static ActorSystem startRemoteActorSystem( * @param logger The logger to output log information. * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying * executor - * @param customConfig Custom Akka config to be combined with the config derived from Flink + * @param customConfig Custom Pekko config to be combined with the config derived from Flink * configuration. * @return The ActorSystem which has been started * @throws Exception Thrown when actor system cannot be started in specified port range @@ -146,7 +146,7 @@ public static ActorSystem startRemoteActorSystem( * @param logger the logger to output log information. * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying * executor - * @param customConfig Custom Akka config to be combined with the config derived from Flink + * @param customConfig Custom Pekko config to be combined with the config derived from Flink * configuration. * @return The ActorSystem which has been started. * @throws Exception @@ -173,18 +173,18 @@ private static ActorSystem startRemoteActorSystem( bindHostPortUrl); try { - Config akkaConfig = - AkkaUtils.getAkkaConfig( + Config pekkoConfig = + PekkoUtils.getConfig( configuration, new HostAndPort(externalAddress, externalPort), new HostAndPort(bindAddress, bindPort), actorSystemExecutorConfiguration); if (customConfig != null) { - akkaConfig = customConfig.withFallback(akkaConfig); + pekkoConfig = customConfig.withFallback(pekkoConfig); } - return startActorSystem(akkaConfig, actorSystemName, logger); + return startActorSystem(pekkoConfig, actorSystemName, logger); } catch (Throwable t) { if (t instanceof ChannelException) { Throwable cause = t.getCause(); @@ -209,7 +209,7 @@ private static ActorSystem startRemoteActorSystem( * @param logger The logger to output log information. * @param actorSystemExecutorConfiguration Configuration for the ActorSystem's underlying * executor. - * @param customConfig Custom Akka config to be combined with the config derived from Flink + * @param customConfig Custom Pekko config to be combined with the config derived from Flink * configuration. * @return The ActorSystem which has been started. * @throws Exception @@ -225,52 +225,41 @@ public static ActorSystem startLocalActorSystem( logger.info("Trying to start local actor system"); try { - Config akkaConfig = - AkkaUtils.getAkkaConfig( + Config pekkoConfig = + PekkoUtils.getConfig( configuration, null, null, actorSystemExecutorConfiguration); if (customConfig != null) { - akkaConfig = customConfig.withFallback(akkaConfig); + pekkoConfig = customConfig.withFallback(pekkoConfig); } - return startActorSystem(akkaConfig, actorSystemName, logger); + return startActorSystem(pekkoConfig, actorSystemName, logger); } catch (Throwable t) { throw new Exception("Could not create actor system", t); } } /** - * Starts an Actor System with given Akka config. + * Starts an Actor System with given Pekko config. * - * @param akkaConfig Config of the started ActorSystem. + * @param config Config of the started ActorSystem. * @param actorSystemName Name of the started ActorSystem. * @param logger The logger to output log information. * @return The ActorSystem which has been started. */ private static ActorSystem startActorSystem( - Config akkaConfig, String actorSystemName, Logger logger) { - logger.debug("Using akka configuration\n {}", akkaConfig); - ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig); + Config config, String actorSystemName, Logger logger) { + logger.debug("Using pekko configuration\n {}", config); + ActorSystem actorSystem = PekkoUtils.createActorSystem(actorSystemName, config); - logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem)); + logger.info("Actor system started at {}", PekkoUtils.getAddress(actorSystem)); return actorSystem; } // ------------------------------------------------------------------------ /** Private constructor to prevent instantiation. */ - private AkkaBootstrapTools() {} - - /** Configuration interface for {@link ActorSystem} underlying executor. */ - public interface ActorSystemExecutorConfiguration { - - /** - * Create the executor {@link Config} for the respective executor. - * - * @return Akka config for the respective executor - */ - Config getAkkaConfig(); - } + private ActorSystemBootstrapTools() {} public static RpcSystem.ForkJoinExecutorConfiguration getForkJoinExecutorConfiguration( final Configuration configuration) { diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/ControlMessages.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ControlMessages.java similarity index 87% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/ControlMessages.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ControlMessages.java index a43327d19fee9..16621ddfdab9b 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/ControlMessages.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/ControlMessages.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; -/** Control messages for the {@link AkkaRpcActor}. */ +/** Control messages for the {@link PekkoRpcActor}. */ public enum ControlMessages { START, // Start processing incoming messages STOP, // Stop processing messages and drop all newly incoming messages - TERMINATE, // Terminate the AkkaRpcActor + TERMINATE, // Terminate the RpcActor } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/CustomSSLEngineProvider.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/CustomSSLEngineProvider.java similarity index 89% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/CustomSSLEngineProvider.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/CustomSSLEngineProvider.java index 7f40c1554aa66..abb0a6f1fad36 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/CustomSSLEngineProvider.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/CustomSSLEngineProvider.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.shaded.netty4.io.netty.handler.ssl.util.FingerprintTrustManagerFactory; -import akka.actor.ActorSystem; -import akka.remote.RemoteTransportException; -import akka.remote.transport.netty.ConfigSSLEngineProvider; import com.typesafe.config.Config; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.remote.RemoteTransportException; +import org.apache.pekko.remote.transport.netty.ConfigSSLEngineProvider; import javax.net.ssl.TrustManager; import javax.net.ssl.TrustManagerFactory; @@ -42,7 +42,7 @@ public class CustomSSLEngineProvider extends ConfigSSLEngineProvider { public CustomSSLEngineProvider(ActorSystem system) { super(system); final Config securityConfig = - system.settings().config().getConfig("akka.remote.classic.netty.ssl.security"); + system.settings().config().getConfig("pekko.remote.classic.netty.ssl.security"); sslTrustStore = securityConfig.getString("trust-store"); sslTrustStorePassword = securityConfig.getString("trust-store-password"); sslCertFingerprints = securityConfig.getStringList("cert-fingerprints"); diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/DeadLettersActor.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/DeadLettersActor.java similarity index 86% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/DeadLettersActor.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/DeadLettersActor.java index 9228f6ccfa2a0..7c0fee856f8dd 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/DeadLettersActor.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/DeadLettersActor.java @@ -16,22 +16,22 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.exceptions.RecipientUnreachableException; import org.apache.flink.runtime.rpc.messages.Message; -import akka.actor.AbstractActor; -import akka.actor.DeadLetter; -import akka.actor.Props; -import akka.actor.Status; -import akka.japi.pf.ReceiveBuilder; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.DeadLetter; +import org.apache.pekko.actor.Props; +import org.apache.pekko.actor.Status; +import org.apache.pekko.japi.pf.ReceiveBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Actor which listens to {@link akka.actor.DeadLetter} and responds with a failure if the message - * was a RPC. + * Actor which listens to {@link org.apache.pekko.actor.DeadLetter} and responds with a failure if + * the message was a RPC. */ public class DeadLettersActor extends AbstractActor { diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/EscalatingSupervisorStrategy.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/EscalatingSupervisorStrategy.java similarity index 83% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/EscalatingSupervisorStrategy.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/EscalatingSupervisorStrategy.java index ccbd25dd2ba98..cccfa5436a7af 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/EscalatingSupervisorStrategy.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/EscalatingSupervisorStrategy.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; -import akka.actor.OneForOneStrategy; -import akka.actor.SupervisorStrategy; -import akka.actor.SupervisorStrategyConfigurator; -import akka.japi.pf.PFBuilder; +import org.apache.pekko.actor.OneForOneStrategy; +import org.apache.pekko.actor.SupervisorStrategy; +import org.apache.pekko.actor.SupervisorStrategyConfigurator; +import org.apache.pekko.japi.pf.PFBuilder; /** Escalating supervisor strategy. */ public class EscalatingSupervisorStrategy implements SupervisorStrategyConfigurator { diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoInvocationHandler.java similarity index 92% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoInvocationHandler.java index 2c2e998f36b61..8cc4e3ffb4c1d 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoInvocationHandler.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.FencedRpcGateway; @@ -26,7 +26,7 @@ import org.apache.flink.runtime.rpc.messages.RemoteFencedMessage; import org.apache.flink.util.Preconditions; -import akka.actor.ActorRef; +import org.apache.pekko.actor.ActorRef; import javax.annotation.Nullable; @@ -37,18 +37,18 @@ import java.util.function.Supplier; /** - * Fenced extension of the {@link AkkaInvocationHandler}. This invocation handler will be used in + * Fenced extension of the {@link PekkoInvocationHandler}. This invocation handler will be used in * combination with the {@link FencedRpcEndpoint}. The fencing is done by wrapping all messages in a * {@link FencedMessage}. * * @param type of the fencing token */ -public class FencedAkkaInvocationHandler extends AkkaInvocationHandler +public class FencedPekkoInvocationHandler extends PekkoInvocationHandler implements MainThreadExecutable, FencedRpcGateway { private final Supplier fencingTokenSupplier; - public FencedAkkaInvocationHandler( + public FencedPekkoInvocationHandler( String address, String hostname, ActorRef rpcEndpoint, diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoRpcActor.java similarity index 92% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoRpcActor.java index 8e034e696b48a..860a263f53976 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/FencedPekkoRpcActor.java @@ -16,31 +16,32 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; import org.apache.flink.runtime.rpc.messages.FencedMessage; import org.apache.flink.runtime.rpc.messages.LocalFencedMessage; +import org.apache.flink.runtime.rpc.pekko.exceptions.UnknownMessageException; import java.io.Serializable; import java.util.Objects; import java.util.concurrent.CompletableFuture; /** - * Fenced extension of the {@link AkkaRpcActor}. This actor will be started for {@link + * Fenced extension of the {@link PekkoRpcActor}. This actor will be started for {@link * FencedRpcEndpoint} and is responsible for filtering out invalid messages with respect to the * current fencing token. * * @param type of the fencing token * @param type of the RpcEndpoint */ -public class FencedAkkaRpcActor & RpcGateway> - extends AkkaRpcActor { +public class FencedPekkoRpcActor< + F extends Serializable, T extends FencedRpcEndpoint & RpcGateway> + extends PekkoRpcActor { - public FencedAkkaRpcActor( + public FencedPekkoRpcActor( T rpcEndpoint, CompletableFuture terminationFuture, int version, @@ -112,7 +113,7 @@ protected void handleRpcMessage(Object message) { } sendErrorIfSender( - new AkkaUnknownMessageException( + new UnknownMessageException( "Unknown message type: Ignoring message " + message + " of type " diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/HostAndPort.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/HostAndPort.java similarity index 96% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/HostAndPort.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/HostAndPort.java index 84b8459c5c24d..f0e8252afbe8f 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/HostAndPort.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/HostAndPort.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.util.Preconditions; diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBasedEndpoint.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoBasedEndpoint.java similarity index 85% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBasedEndpoint.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoBasedEndpoint.java index eff74d63506c7..f3c8ec0a3a253 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaBasedEndpoint.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoBasedEndpoint.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.RpcGateway; -import akka.actor.ActorRef; +import org.apache.pekko.actor.ActorRef; -/** Interface for Akka based rpc gateways. */ -interface AkkaBasedEndpoint extends RpcGateway { +/** Interface for Pekko based rpc gateways. */ +interface PekkoBasedEndpoint extends RpcGateway { /** * Returns the {@link ActorRef} of the underlying RPC actor. diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoInvocationHandler.java similarity index 92% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoInvocationHandler.java index 3cd3c03c134cd..bc117df44b712 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoInvocationHandler.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.AkkaOptions; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.Local; import org.apache.flink.runtime.rpc.MainThreadExecutable; @@ -37,8 +37,9 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; -import akka.actor.ActorRef; -import akka.pattern.Patterns; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.pattern.AskTimeoutException; +import org.apache.pekko.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,15 +64,15 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Invocation handler to be used with an {@link AkkaRpcActor}. The invocation handler wraps the rpc - * in an {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is + * Invocation handler to be used with an {@link PekkoRpcActor}. The invocation handler wraps the rpc + * in an {@link RpcInvocation} message and then sends it to the {@link PekkoRpcActor} where it is * executed. */ -class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, RpcServer { - private static final Logger LOG = LoggerFactory.getLogger(AkkaInvocationHandler.class); +class PekkoInvocationHandler implements InvocationHandler, PekkoBasedEndpoint, RpcServer { + private static final Logger LOG = LoggerFactory.getLogger(PekkoInvocationHandler.class); /** - * The Akka (RPC) address of {@link #rpcEndpoint} including host and port of the ActorSystem in + * The Pekko (RPC) address of {@link #rpcEndpoint} including host and port of the ActorSystem in * which the actor is running. */ private final String address; @@ -97,7 +98,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaBasedEndpoint, Rpc private final boolean captureAskCallStack; - AkkaInvocationHandler( + PekkoInvocationHandler( String address, String hostname, ActorRef rpcEndpoint, @@ -126,7 +127,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Object result; - if (declaringClass.equals(AkkaBasedEndpoint.class) + if (declaringClass.equals(PekkoBasedEndpoint.class) || declaringClass.equals(Object.class) || declaringClass.equals(RpcGateway.class) || declaringClass.equals(StartStoppable.class) @@ -135,7 +136,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl result = method.invoke(this, args); } else if (declaringClass.equals(FencedRpcGateway.class)) { throw new UnsupportedOperationException( - "AkkaInvocationHandler does not support the call FencedRpcGateway#" + "InvocationHandler does not support the call FencedRpcGateway#" + method.getName() + ". This indicates that you retrieved a FencedRpcGateway without specifying a " + "fencing token. Please use RpcService#connect(RpcService, F, Time) with F being the fencing token to " @@ -338,7 +339,7 @@ protected void tell(Object message) { */ protected CompletableFuture ask(Object message, Duration timeout) { final CompletableFuture response = - AkkaFutureUtils.toJava(Patterns.ask(rpcEndpoint, message, timeout.toMillis())); + ScalaFutureUtils.toJava(Patterns.ask(rpcEndpoint, message, timeout.toMillis())); return guardCompletionWithContextClassLoader(response, flinkClassLoader); } @@ -359,9 +360,9 @@ public CompletableFuture getTerminationFuture() { private static Object deserializeValueIfNeeded( Object o, Method method, ClassLoader flinkClassLoader) { - if (o instanceof AkkaRpcSerializedValue) { + if (o instanceof RpcSerializedValue) { try { - return ((AkkaRpcSerializedValue) o).deserializeValue(flinkClassLoader); + return ((RpcSerializedValue) o).deserializeValue(flinkClassLoader); } catch (IOException | ClassNotFoundException e) { throw new CompletionException( new RpcException( @@ -379,13 +380,13 @@ static Throwable resolveTimeoutException( @Nullable Throwable callStackCapture, String recipient, RpcInvocation rpcInvocation) { - if (!(exception instanceof akka.pattern.AskTimeoutException)) { + if (!(exception instanceof AskTimeoutException)) { return exception; } final Exception newException; - if (AkkaRpcServiceUtils.isRecipientTerminatedException(exception)) { + if (PekkoRpcServiceUtils.isRecipientTerminatedException(exception)) { newException = new RecipientUnreachableException( "unknown", recipient, rpcInvocation.toString()); @@ -393,7 +394,7 @@ static Throwable resolveTimeoutException( newException = new TimeoutException( String.format( - "Invocation of [%s] at recipient [%s] timed out. This is usually caused by: 1) Akka failed sending " + "Invocation of [%s] at recipient [%s] timed out. This is usually caused by: 1) Pekko failed sending " + "the message silently, due to problems like oversized payload or serialization failures. " + "In that case, you should find detailed error information in the logs. 2) The recipient needs " + "more time for responding, due to problems like slow machines or network jitters. In that case, you can try to increase %s.", diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActor.java similarity index 86% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActor.java index d51f802e5a28e..f3724fba0e5cc 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActor.java @@ -16,33 +16,33 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.Local; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaHandshakeException; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcInvalidStateException; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; import org.apache.flink.runtime.rpc.exceptions.EndpointNotStartedException; +import org.apache.flink.runtime.rpc.exceptions.HandshakeException; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; +import org.apache.flink.runtime.rpc.exceptions.RpcException; import org.apache.flink.runtime.rpc.messages.CallAsync; import org.apache.flink.runtime.rpc.messages.HandshakeSuccessMessage; import org.apache.flink.runtime.rpc.messages.RemoteHandshakeMessage; import org.apache.flink.runtime.rpc.messages.RpcInvocation; import org.apache.flink.runtime.rpc.messages.RunAsync; +import org.apache.flink.runtime.rpc.pekko.exceptions.RpcInvalidStateException; +import org.apache.flink.runtime.rpc.pekko.exceptions.UnknownMessageException; import org.apache.flink.types.Either; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.FutureUtils; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.Status; -import akka.japi.pf.ReceiveBuilder; -import akka.pattern.Patterns; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.Status; +import org.apache.pekko.japi.pf.ReceiveBuilder; +import org.apache.pekko.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +65,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} + * Pekko rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} * {@link ControlMessages} messages. * *

The {@link RpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint} @@ -74,14 +74,14 @@ *

The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed * in the context of the actor thread. * - *

The {@link ControlMessages} message controls the processing behaviour of the akka rpc actor. A - * {@link ControlMessages#START} starts processing incoming messages. A {@link ControlMessages#STOP} - * message stops processing messages. All messages which arrive when the processing is stopped, will - * be discarded. + *

The {@link ControlMessages} message controls the processing behaviour of the pekko rpc actor. + * A {@link ControlMessages#START} starts processing incoming messages. A {@link + * ControlMessages#STOP} message stops processing messages. All messages which arrive when the + * processing is stopped, will be discarded. * * @param Type of the {@link RpcEndpoint} */ -class AkkaRpcActor extends AbstractActor { +class PekkoRpcActor extends AbstractActor { protected final Logger log = LoggerFactory.getLogger(getClass()); @@ -107,7 +107,7 @@ class AkkaRpcActor extends AbstractActor { @Nonnull private State state; - AkkaRpcActor( + PekkoRpcActor( final T rpcEndpoint, final CompletableFuture terminationFuture, final int version, @@ -126,7 +126,7 @@ class AkkaRpcActor extends AbstractActor { this.rpcEndpointStopped = new AtomicBoolean(false); this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure( - new AkkaRpcException( + new RpcException( String.format( "RpcEndpoint %s has not been properly stopped.", rpcEndpoint.getEndpointId()))); @@ -210,7 +210,7 @@ private void handleUnknownControlMessage(ControlMessages controlMessage) { "Received unknown control message %s. Dropping this message!", controlMessage); log.warn(message); - sendErrorIfSender(new AkkaUnknownMessageException(message)); + sendErrorIfSender(new UnknownMessageException(message)); } protected void handleRpcMessage(Object message) { @@ -227,7 +227,7 @@ protected void handleRpcMessage(Object message) { message); sendErrorIfSender( - new AkkaUnknownMessageException( + new UnknownMessageException( "Received unknown message " + message + " of type " @@ -239,13 +239,13 @@ protected void handleRpcMessage(Object message) { private void handleHandshakeMessage(RemoteHandshakeMessage handshakeMessage) { if (!isCompatibleVersion(handshakeMessage.getVersion())) { sendErrorIfSender( - new AkkaHandshakeException( + new HandshakeException( String.format( "Version mismatch between source (%s) and target (%s) rpc component. Please verify that all components have the same version.", handshakeMessage.getVersion(), getVersion()))); } else if (!isGatewaySupported(handshakeMessage.getRpcGateway())) { sendErrorIfSender( - new AkkaHandshakeException( + new HandshakeException( String.format( "The rpc endpoint does not support the gateway %s.", handshakeMessage.getRpcGateway().getSimpleName()))); @@ -340,7 +340,7 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { private void sendSyncResponse( Object response, String methodName, boolean isLocalRpcInvocation) { if (isRemoteSender(getSender()) || (forceSerialization && !isLocalRpcInvocation)) { - Either serializedResult = + Either serializedResult = serializeRemoteResultAndVerifySize(response, methodName); if (serializedResult.isLeft()) { @@ -366,10 +366,8 @@ private void sendAsyncResponse( } else { if (isRemoteSender(sender) || (forceSerialization && !isLocalRpcInvocation)) { - Either - serializedResult = - serializeRemoteResultAndVerifySize( - value, methodName); + Either serializedResult = + serializeRemoteResultAndVerifySize(value, methodName); if (serializedResult.isLeft()) { promise.success(serializedResult.left()); @@ -392,15 +390,15 @@ private boolean isRemoteSender(ActorRef sender) { return !sender.path().address().hasLocalScope(); } - private Either serializeRemoteResultAndVerifySize( + private Either serializeRemoteResultAndVerifySize( Object result, String methodName) { try { - AkkaRpcSerializedValue serializedResult = AkkaRpcSerializedValue.valueOf(result); + RpcSerializedValue serializedResult = RpcSerializedValue.valueOf(result); long resultSize = serializedResult.getSerializedDataLength(); if (resultSize > maximumFramesize) { return Either.Right( - new AkkaRpcException( + new RpcException( "The method " + methodName + "'s result size " @@ -413,7 +411,7 @@ private Either serializeRemoteResultAn } } catch (IOException e) { return Either.Right( - new AkkaRpcException( + new RpcException( "Failed to serialize the result for RPC call : " + methodName + '.', e)); } @@ -523,18 +521,16 @@ private void stop(RpcEndpointTerminationResult rpcEndpointTerminationResult) { // --------------------------------------------------------------------------- interface State { - default State start(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { - throw new AkkaRpcInvalidStateException( - invalidStateTransitionMessage(StartedState.STARTED)); + default State start(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { + throw new RpcInvalidStateException(invalidStateTransitionMessage(StartedState.STARTED)); } default State stop() { - throw new AkkaRpcInvalidStateException( - invalidStateTransitionMessage(StoppedState.STOPPED)); + throw new RpcInvalidStateException(invalidStateTransitionMessage(StoppedState.STOPPED)); } - default State terminate(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { - throw new AkkaRpcInvalidStateException( + default State terminate(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { + throw new RpcInvalidStateException( invalidStateTransitionMessage(TerminatingState.TERMINATING)); } @@ -548,7 +544,7 @@ default boolean isRunning() { default String invalidStateTransitionMessage(State targetState) { return String.format( - "AkkaRpcActor is currently in state %s and cannot go into state %s.", + "RpcActor is currently in state %s and cannot go into state %s.", this, targetState); } } @@ -558,7 +554,7 @@ enum StartedState implements State { STARTED; @Override - public State start(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { + public State start(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { return STARTED; } @@ -568,25 +564,25 @@ public State stop() { } @Override - public State terminate(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { - akkaRpcActor.mainThreadValidator.enterMainThread(); + public State terminate(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { + pekkoRpcActor.mainThreadValidator.enterMainThread(); CompletableFuture terminationFuture; try { terminationFuture = runWithContextClassLoader( - () -> akkaRpcActor.rpcEndpoint.internalCallOnStop(), + () -> pekkoRpcActor.rpcEndpoint.internalCallOnStop(), flinkClassLoader); } catch (Throwable t) { terminationFuture = FutureUtils.completedExceptionally( - new AkkaRpcException( + new RpcException( String.format( "Failure while stopping RpcEndpoint %s.", - akkaRpcActor.rpcEndpoint.getEndpointId()), + pekkoRpcActor.rpcEndpoint.getEndpointId()), t)); } finally { - akkaRpcActor.mainThreadValidator.exitMainThread(); + pekkoRpcActor.mainThreadValidator.exitMainThread(); } // IMPORTANT: This only works if we don't use a restarting supervisor strategy. @@ -598,7 +594,7 @@ public State terminate(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoade terminationFuture.whenComplete( (ignored, throwable) -> - akkaRpcActor.stop(RpcEndpointTerminationResult.of(throwable))); + pekkoRpcActor.stop(RpcEndpointTerminationResult.of(throwable))); return TerminatingState.TERMINATING; } @@ -614,22 +610,22 @@ enum StoppedState implements State { STOPPED; @Override - public State start(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { - akkaRpcActor.mainThreadValidator.enterMainThread(); + public State start(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { + pekkoRpcActor.mainThreadValidator.enterMainThread(); try { runWithContextClassLoader( - () -> akkaRpcActor.rpcEndpoint.internalCallOnStart(), flinkClassLoader); + () -> pekkoRpcActor.rpcEndpoint.internalCallOnStart(), flinkClassLoader); } catch (Throwable throwable) { - akkaRpcActor.stop( + pekkoRpcActor.stop( RpcEndpointTerminationResult.failure( - new AkkaRpcException( + new RpcException( String.format( "Could not start RpcEndpoint %s.", - akkaRpcActor.rpcEndpoint.getEndpointId()), + pekkoRpcActor.rpcEndpoint.getEndpointId()), throwable))); } finally { - akkaRpcActor.mainThreadValidator.exitMainThread(); + pekkoRpcActor.mainThreadValidator.exitMainThread(); } return StartedState.STARTED; @@ -641,8 +637,8 @@ public State stop() { } @Override - public State terminate(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { - akkaRpcActor.stop(RpcEndpointTerminationResult.success()); + public State terminate(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { + pekkoRpcActor.stop(RpcEndpointTerminationResult.success()); return TerminatingState.TERMINATING; } @@ -653,7 +649,7 @@ enum TerminatingState implements State { TERMINATING; @Override - public State terminate(AkkaRpcActor akkaRpcActor, ClassLoader flinkClassLoader) { + public State terminate(PekkoRpcActor pekkoRpcActor, ClassLoader flinkClassLoader) { return TERMINATING; } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcService.java similarity index 82% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcService.java index c9a16c451eead..c9d276685edcb 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcService.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.concurrent.akka.ActorSystemScheduledExecutorAdapter; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ActorSystemScheduledExecutorAdapter; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; @@ -29,8 +29,8 @@ import org.apache.flink.runtime.rpc.RpcServer; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcRuntimeException; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; +import org.apache.flink.runtime.rpc.exceptions.RpcRuntimeException; import org.apache.flink.runtime.rpc.messages.HandshakeSuccessMessage; import org.apache.flink.runtime.rpc.messages.RemoteHandshakeMessage; import org.apache.flink.util.AutoCloseableAsync; @@ -40,14 +40,14 @@ import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ScheduledExecutor; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.ActorSelection; -import akka.actor.ActorSystem; -import akka.actor.Address; -import akka.actor.DeadLetter; -import akka.actor.Props; -import akka.pattern.Patterns; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSelection; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Address; +import org.apache.pekko.actor.DeadLetter; +import org.apache.pekko.actor.Props; +import org.apache.pekko.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,20 +80,20 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * Akka based {@link RpcService} implementation. The RPC service starts an Akka actor to receive RPC + * Pekko based {@link RpcService} implementation. The RPC service starts an actor to receive RPC * invocations from a {@link RpcGateway}. */ @ThreadSafe -public class AkkaRpcService implements RpcService { +public class PekkoRpcService implements RpcService { - private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class); + private static final Logger LOG = LoggerFactory.getLogger(PekkoRpcService.class); static final int VERSION = 2; private final Object lock = new Object(); private final ActorSystem actorSystem; - private final AkkaRpcServiceConfiguration configuration; + private final PekkoRpcServiceConfiguration configuration; private final ClassLoader flinkClassLoader; @@ -114,20 +114,20 @@ public class AkkaRpcService implements RpcService { private volatile boolean stopped; @VisibleForTesting - public AkkaRpcService( - final ActorSystem actorSystem, final AkkaRpcServiceConfiguration configuration) { - this(actorSystem, configuration, AkkaRpcService.class.getClassLoader()); + public PekkoRpcService( + final ActorSystem actorSystem, final PekkoRpcServiceConfiguration configuration) { + this(actorSystem, configuration, PekkoRpcService.class.getClassLoader()); } - AkkaRpcService( + PekkoRpcService( final ActorSystem actorSystem, - final AkkaRpcServiceConfiguration configuration, + final PekkoRpcServiceConfiguration configuration, final ClassLoader flinkClassLoader) { this.actorSystem = checkNotNull(actorSystem, "actor system"); - this.configuration = checkNotNull(configuration, "akka rpc service configuration"); + this.configuration = checkNotNull(configuration, "pekko rpc service configuration"); this.flinkClassLoader = checkNotNull(flinkClassLoader, "flinkClassLoader"); - Address actorSystemAddress = AkkaUtils.getAddress(actorSystem); + Address actorSystemAddress = PekkoUtils.getAddress(actorSystem); if (actorSystemAddress.host().isDefined()) { address = actorSystemAddress.host().get(); @@ -143,7 +143,7 @@ public AkkaRpcService( captureAskCallstacks = configuration.captureAskCallStack(); - // Akka always sets the threads context class loader to the class loader with which it was + // Pekko always sets the threads context class loader to the class loader with which it was // loaded (i.e., the plugin class loader) // we must ensure that the context class loader is set to the Flink class loader when we // call into Flink @@ -170,7 +170,7 @@ private Supervisor startSupervisorActor() { final ExecutorService terminationFutureExecutor = Executors.newSingleThreadExecutor( new ExecutorThreadFactory( - "AkkaRpcService-Supervisor-Termination-Future-Executor")); + "RpcService-Supervisor-Termination-Future-Executor")); final ActorRef actorRef = SupervisorActor.startSupervisorActor( actorSystem, @@ -222,7 +222,7 @@ public CompletableFuture connect( (ActorRef actorRef) -> { Tuple2 addressHostname = extractAddressHostname(actorRef); - return new AkkaInvocationHandler( + return new PekkoInvocationHandler( addressHostname.f0, addressHostname.f1, actorRef, @@ -245,7 +245,7 @@ public > CompletableFuture (ActorRef actorRef) -> { Tuple2 addressHostname = extractAddressHostname(actorRef); - return new FencedAkkaInvocationHandler<>( + return new FencedPekkoInvocationHandler<>( addressHostname.f0, addressHostname.f1, actorRef, @@ -263,8 +263,7 @@ public > CompletableFuture public RpcServer startServer(C rpcEndpoint) { checkNotNull(rpcEndpoint, "rpc endpoint"); - final SupervisorActor.ActorRegistration actorRegistration = - registerAkkaRpcActor(rpcEndpoint); + final SupervisorActor.ActorRegistration actorRegistration = registerRpcActor(rpcEndpoint); final ActorRef actorRef = actorRegistration.getActorRef(); final CompletableFuture actorTerminationFuture = actorRegistration.getTerminationFuture(); @@ -274,7 +273,7 @@ public RpcServer startServer(C rpcEndpoint) rpcEndpoint.getClass().getName(), actorRef.path()); - final String akkaAddress = AkkaUtils.getAkkaURL(actorSystem, actorRef); + final String address = PekkoUtils.getRpcURL(actorSystem, actorRef); final String hostname; Option host = actorRef.path().address().host(); if (host.isEmpty()) { @@ -287,15 +286,15 @@ public RpcServer startServer(C rpcEndpoint) new HashSet<>(RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass())); implementedRpcGateways.add(RpcServer.class); - implementedRpcGateways.add(AkkaBasedEndpoint.class); + implementedRpcGateways.add(PekkoBasedEndpoint.class); - final InvocationHandler akkaInvocationHandler; + final InvocationHandler invocationHandler; if (rpcEndpoint instanceof FencedRpcEndpoint) { - // a FencedRpcEndpoint needs a FencedAkkaInvocationHandler - akkaInvocationHandler = - new FencedAkkaInvocationHandler<>( - akkaAddress, + // a FencedRpcEndpoint needs a FencedPekkoInvocationHandler + invocationHandler = + new FencedPekkoInvocationHandler<>( + address, hostname, actorRef, configuration.getTimeout(), @@ -306,9 +305,9 @@ public RpcServer startServer(C rpcEndpoint) captureAskCallstacks, flinkClassLoader); } else { - akkaInvocationHandler = - new AkkaInvocationHandler( - akkaAddress, + invocationHandler = + new PekkoInvocationHandler( + address, hostname, actorRef, configuration.getTimeout(), @@ -331,30 +330,30 @@ public RpcServer startServer(C rpcEndpoint) classLoader, implementedRpcGateways.toArray( new Class[implementedRpcGateways.size()]), - akkaInvocationHandler); + invocationHandler); return server; } - private - SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) { - final Class akkaRpcActorType; + private SupervisorActor.ActorRegistration registerRpcActor( + C rpcEndpoint) { + final Class rpcActorType; if (rpcEndpoint instanceof FencedRpcEndpoint) { - akkaRpcActorType = FencedAkkaRpcActor.class; + rpcActorType = FencedPekkoRpcActor.class; } else { - akkaRpcActorType = AkkaRpcActor.class; + rpcActorType = PekkoRpcActor.class; } synchronized (lock) { checkState(!stopped, "RpcService is stopped"); - final SupervisorActor.StartAkkaRpcActorResponse startAkkaRpcActorResponse = - SupervisorActor.startAkkaRpcActor( + final SupervisorActor.StartRpcActorResponse startRpcActorResponse = + SupervisorActor.startRpcActor( supervisor.getActor(), actorTerminationFuture -> Props.create( - akkaRpcActorType, + rpcActorType, rpcEndpoint, actorTerminationFuture, getVersion(), @@ -364,12 +363,12 @@ SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) { rpcEndpoint.getEndpointId()); final SupervisorActor.ActorRegistration actorRegistration = - startAkkaRpcActorResponse.orElseThrow( + startRpcActorResponse.orElseThrow( cause -> - new AkkaRpcRuntimeException( + new RpcRuntimeException( String.format( "Could not create the %s for %s.", - AkkaRpcActor.class.getSimpleName(), + PekkoRpcActor.class.getSimpleName(), rpcEndpoint.getEndpointId()), cause)); @@ -381,20 +380,20 @@ SupervisorActor.ActorRegistration registerAkkaRpcActor(C rpcEndpoint) { @Override public void stopServer(RpcServer selfGateway) { - if (selfGateway instanceof AkkaBasedEndpoint) { - final AkkaBasedEndpoint akkaClient = (AkkaBasedEndpoint) selfGateway; + if (selfGateway instanceof PekkoBasedEndpoint) { + final PekkoBasedEndpoint client = (PekkoBasedEndpoint) selfGateway; final RpcEndpoint rpcEndpoint; synchronized (lock) { if (stopped) { return; } else { - rpcEndpoint = actors.remove(akkaClient.getActorRef()); + rpcEndpoint = actors.remove(client.getActorRef()); } } if (rpcEndpoint != null) { - terminateAkkaRpcActor(akkaClient.getActorRef(), rpcEndpoint); + terminateRpcActor(client.getActorRef(), rpcEndpoint); } else { LOG.debug( "RPC endpoint {} already stopped or from different RPC service", @@ -405,28 +404,27 @@ public void stopServer(RpcServer selfGateway) { @Override public CompletableFuture closeAsync() { - final CompletableFuture akkaRpcActorsTerminationFuture; + final CompletableFuture rpcActorsTerminationFuture; synchronized (lock) { if (stopped) { return terminationFuture; } - LOG.info("Stopping Akka RPC service."); + LOG.info("Stopping Pekko RPC service."); stopped = true; - akkaRpcActorsTerminationFuture = terminateAkkaRpcActors(); + rpcActorsTerminationFuture = terminateRpcActors(); } final CompletableFuture supervisorTerminationFuture = - FutureUtils.composeAfterwards( - akkaRpcActorsTerminationFuture, supervisor::closeAsync); + FutureUtils.composeAfterwards(rpcActorsTerminationFuture, supervisor::closeAsync); final CompletableFuture actorSystemTerminationFuture = FutureUtils.composeAfterwards( supervisorTerminationFuture, - () -> AkkaFutureUtils.toJava(actorSystem.terminate())); + () -> ScalaFutureUtils.toJava(actorSystem.terminate())); actorSystemTerminationFuture.whenComplete( (Void ignored, Throwable throwable) -> { @@ -434,7 +432,7 @@ public CompletableFuture closeAsync() { () -> FutureUtils.doForward(ignored, throwable, terminationFuture), flinkClassLoader); - LOG.info("Stopped Akka RPC service."); + LOG.info("Stopped Pekko RPC service."); }); return terminationFuture; @@ -442,24 +440,24 @@ public CompletableFuture closeAsync() { @GuardedBy("lock") @Nonnull - private CompletableFuture terminateAkkaRpcActors() { - final Collection> akkaRpcActorTerminationFutures = + private CompletableFuture terminateRpcActors() { + final Collection> rpcActorTerminationFutures = new ArrayList<>(actors.size()); for (Map.Entry actorRefRpcEndpointEntry : actors.entrySet()) { - akkaRpcActorTerminationFutures.add( - terminateAkkaRpcActor( + rpcActorTerminationFutures.add( + terminateRpcActor( actorRefRpcEndpointEntry.getKey(), actorRefRpcEndpointEntry.getValue())); } actors.clear(); - return FutureUtils.waitForAll(akkaRpcActorTerminationFutures); + return FutureUtils.waitForAll(rpcActorTerminationFutures); } - private CompletableFuture terminateAkkaRpcActor( - ActorRef akkaRpcActorRef, RpcEndpoint rpcEndpoint) { - akkaRpcActorRef.tell(ControlMessages.TERMINATE, ActorRef.noSender()); + private CompletableFuture terminateRpcActor( + ActorRef rpcActorRef, RpcEndpoint rpcEndpoint) { + rpcActorRef.tell(ControlMessages.TERMINATE, ActorRef.noSender()); return rpcEndpoint.getTerminationFuture(); } @@ -474,7 +472,7 @@ public ScheduledExecutor getScheduledExecutor() { // --------------------------------------------------------------------------------------- private Tuple2 extractAddressHostname(ActorRef actorRef) { - final String actorAddress = AkkaUtils.getAkkaURL(actorSystem, actorRef); + final String actorAddress = PekkoUtils.getRpcURL(actorSystem, actorRef); final String hostname; Option host = actorRef.path().address().host(); if (host.isEmpty()) { @@ -502,7 +500,7 @@ private CompletableFuture connectInternal( final CompletableFuture handshakeFuture = actorRefFuture.thenCompose( (ActorRef actorRef) -> - AkkaFutureUtils.toJava( + ScalaFutureUtils.toJava( Patterns.ask( actorRef, new RemoteHandshakeMessage( diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceConfiguration.java similarity index 85% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceConfiguration.java index 41a87dd4ec9d6..e52c277b37b4a 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceConfiguration.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -26,8 +26,8 @@ import static org.apache.flink.util.Preconditions.checkArgument; -/** Configuration for the {@link AkkaRpcService}. */ -public class AkkaRpcServiceConfiguration { +/** Configuration for the {@link PekkoRpcService}. */ +public class PekkoRpcServiceConfiguration { @Nonnull private final Configuration configuration; @@ -39,7 +39,7 @@ public class AkkaRpcServiceConfiguration { private final boolean forceRpcInvocationSerialization; - private AkkaRpcServiceConfiguration( + private PekkoRpcServiceConfiguration( @Nonnull Configuration configuration, @Nonnull Duration timeout, long maximumFramesize, @@ -76,17 +76,17 @@ public boolean isForceRpcInvocationSerialization() { return forceRpcInvocationSerialization; } - public static AkkaRpcServiceConfiguration fromConfiguration(Configuration configuration) { + public static PekkoRpcServiceConfiguration fromConfiguration(Configuration configuration) { final Duration timeout = configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION); - final long maximumFramesize = AkkaRpcServiceUtils.extractMaximumFramesize(configuration); + final long maximumFramesize = PekkoRpcServiceUtils.extractMaximumFramesize(configuration); final boolean captureAskCallStacks = configuration.get(AkkaOptions.CAPTURE_ASK_CALLSTACK); final boolean forceRpcInvocationSerialization = AkkaOptions.isForceRpcInvocationSerializationEnabled(configuration); - return new AkkaRpcServiceConfiguration( + return new PekkoRpcServiceConfiguration( configuration, timeout, maximumFramesize, @@ -94,7 +94,7 @@ public static AkkaRpcServiceConfiguration fromConfiguration(Configuration config forceRpcInvocationSerialization); } - public static AkkaRpcServiceConfiguration defaultConfiguration() { + public static PekkoRpcServiceConfiguration defaultConfiguration() { return fromConfiguration(new Configuration()); } } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceUtils.java similarity index 75% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceUtils.java index 2e91569d933e9..4b45416619594 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; @@ -30,9 +30,9 @@ import org.apache.flink.util.TemporaryClassLoaderContext; import org.apache.flink.util.function.TriFunction; -import akka.actor.ActorSystem; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; +import org.apache.pekko.actor.ActorSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,60 +50,60 @@ * These RPC utilities contain helper methods around RPC use, such as starting an RPC service, or * constructing RPC addresses. */ -public class AkkaRpcServiceUtils { +public class PekkoRpcServiceUtils { - private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcServiceUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(PekkoRpcServiceUtils.class); - private static final String AKKA_TCP = "akka.tcp"; - private static final String AKKA_SSL_TCP = "akka.ssl.tcp"; + private static final String PEKKO_TCP = "pekko.tcp"; + private static final String PEKKO_SSL_TCP = "pekko.ssl.tcp"; static final String SUPERVISOR_NAME = "rpc"; - private static final String SIMPLE_AKKA_CONFIG_TEMPLATE = - "akka {remote.classic {netty.tcp {maximum-frame-size = %s}}}"; + private static final String SIMPLE_CONFIG_TEMPLATE = + "pekko {remote.classic {netty.tcp {maximum-frame-size = %s}}}"; private static final String MAXIMUM_FRAME_SIZE_PATH = - "akka.remote.classic.netty.tcp.maximum-frame-size"; + "pekko.remote.classic.netty.tcp.maximum-frame-size"; // ------------------------------------------------------------------------ // RPC instantiation // ------------------------------------------------------------------------ - static AkkaRpcService createRemoteRpcService( + static PekkoRpcService createRemoteRpcService( Configuration configuration, @Nullable String externalAddress, String externalPortRange, @Nullable String bindAddress, @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort) throws Exception { - final AkkaRpcServiceBuilder akkaRpcServiceBuilder = - AkkaRpcServiceUtils.remoteServiceBuilder( + final PekkoRpcServiceBuilder rpcServiceBuilder = + PekkoRpcServiceUtils.remoteServiceBuilder( configuration, externalAddress, externalPortRange); if (bindAddress != null) { - akkaRpcServiceBuilder.withBindAddress(bindAddress); + rpcServiceBuilder.withBindAddress(bindAddress); } - bindPort.ifPresent(akkaRpcServiceBuilder::withBindPort); + bindPort.ifPresent(rpcServiceBuilder::withBindPort); - return akkaRpcServiceBuilder.createAndStart(); + return rpcServiceBuilder.createAndStart(); } - static AkkaRpcServiceBuilder remoteServiceBuilder( + static PekkoRpcServiceBuilder remoteServiceBuilder( Configuration configuration, @Nullable String externalAddress, String externalPortRange) { - return new AkkaRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange); + return new PekkoRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange); } @VisibleForTesting - static AkkaRpcServiceBuilder remoteServiceBuilder( + static PekkoRpcServiceBuilder remoteServiceBuilder( Configuration configuration, @Nullable String externalAddress, int externalPort) { return remoteServiceBuilder(configuration, externalAddress, String.valueOf(externalPort)); } - static AkkaRpcServiceBuilder localServiceBuilder(Configuration configuration) { - return new AkkaRpcServiceBuilder(configuration, LOG); + static PekkoRpcServiceBuilder localServiceBuilder(Configuration configuration) { + return new PekkoRpcServiceBuilder(configuration, LOG); } // ------------------------------------------------------------------------ @@ -138,7 +138,7 @@ public static String getRpcUrl( port, endpointName, addressResolution, - sslEnabled ? AkkaProtocol.SSL_TCP : AkkaProtocol.TCP); + sslEnabled ? Protocol.SSL_TCP : Protocol.TCP); } /** @@ -147,7 +147,7 @@ public static String getRpcUrl( * @param endpointName The name of the RPC endpoint. * @param addressResolution Whether to try address resolution of the given hostname or not. This * allows to fail fast in case that the hostname cannot be resolved. - * @param akkaProtocol True, if security/encryption is enabled, false otherwise. + * @param protocol True, if security/encryption is enabled, false otherwise. * @return The RPC URL of the specified RPC endpoint. */ public static String getRpcUrl( @@ -155,7 +155,7 @@ public static String getRpcUrl( int port, String endpointName, AddressResolution addressResolution, - AkkaProtocol akkaProtocol) + Protocol protocol) throws UnknownHostException { checkNotNull(hostname, "hostname is null"); @@ -171,7 +171,7 @@ public static String getRpcUrl( final String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(hostname, port); return internalRpcUrl( - endpointName, Optional.of(new RemoteAddressInformation(hostPort, akkaProtocol))); + endpointName, Optional.of(new RemoteAddressInformation(hostPort, protocol))); } public static String getLocalRpcUrl(String endpointName) { @@ -184,19 +184,19 @@ public static boolean isRecipientTerminatedException(Throwable exception) { private static final class RemoteAddressInformation { private final String hostnameAndPort; - private final AkkaProtocol akkaProtocol; + private final Protocol protocol; - private RemoteAddressInformation(String hostnameAndPort, AkkaProtocol akkaProtocol) { + private RemoteAddressInformation(String hostnameAndPort, Protocol protocol) { this.hostnameAndPort = hostnameAndPort; - this.akkaProtocol = akkaProtocol; + this.protocol = protocol; } private String getHostnameAndPort() { return hostnameAndPort; } - private AkkaProtocol getAkkaProtocol() { - return akkaProtocol; + private Protocol getProtocol() { + return protocol; } } @@ -204,8 +204,8 @@ private static String internalRpcUrl( String endpointName, Optional remoteAddressInformation) { final String protocolPrefix = remoteAddressInformation - .map(rai -> akkaProtocolToString(rai.getAkkaProtocol())) - .orElse("akka"); + .map(rai -> protocolToString(rai.getProtocol())) + .orElse("pekko"); final Optional optionalHostnameAndPort = remoteAddressInformation.map(RemoteAddressInformation::getHostnameAndPort); @@ -218,12 +218,12 @@ private static String internalRpcUrl( return url.toString(); } - private static String akkaProtocolToString(AkkaProtocol akkaProtocol) { - return akkaProtocol == AkkaProtocol.SSL_TCP ? AKKA_SSL_TCP : AKKA_TCP; + private static String protocolToString(Protocol protocol) { + return protocol == Protocol.SSL_TCP ? PEKKO_SSL_TCP : PEKKO_TCP; } - /** Whether to use TCP or encrypted TCP for Akka. */ - public enum AkkaProtocol { + /** Whether to use TCP or encrypted TCP for Pekko. */ + public enum Protocol { TCP, SSL_TCP } @@ -234,24 +234,24 @@ public enum AkkaProtocol { public static long extractMaximumFramesize(Configuration configuration) { String maxFrameSizeStr = configuration.getString(AkkaOptions.FRAMESIZE); - String akkaConfigStr = String.format(SIMPLE_AKKA_CONFIG_TEMPLATE, maxFrameSizeStr); - Config akkaConfig = ConfigFactory.parseString(akkaConfigStr); - return akkaConfig.getBytes(MAXIMUM_FRAME_SIZE_PATH); + String configStr = String.format(SIMPLE_CONFIG_TEMPLATE, maxFrameSizeStr); + Config config = ConfigFactory.parseString(configStr); + return config.getBytes(MAXIMUM_FRAME_SIZE_PATH); } // ------------------------------------------------------------------------ // RPC service builder // ------------------------------------------------------------------------ - /** Builder for {@link AkkaRpcService}. */ - static class AkkaRpcServiceBuilder implements RpcSystem.RpcServiceBuilder { + /** Builder for {@link PekkoRpcService}. */ + static class PekkoRpcServiceBuilder implements RpcSystem.RpcServiceBuilder { private final Configuration configuration; private final Logger logger; @Nullable private final String externalAddress; @Nullable private final String externalPortRange; - private String actorSystemName = AkkaUtils.getFlinkActorSystemName(); + private String actorSystemName = PekkoUtils.getFlinkActorSystemName(); @Nullable private Config actorSystemExecutorConfiguration = null; @@ -260,7 +260,7 @@ static class AkkaRpcServiceBuilder implements RpcSystem.RpcServiceBuilder { @Nullable private Integer bindPort = null; /** Builder for creating a remote RPC service. */ - private AkkaRpcServiceBuilder( + private PekkoRpcServiceBuilder( final Configuration configuration, final Logger logger, @Nullable final String externalAddress, @@ -275,7 +275,7 @@ private AkkaRpcServiceBuilder( } /** Builder for creating a local RPC service. */ - private AkkaRpcServiceBuilder(final Configuration configuration, final Logger logger) { + private PekkoRpcServiceBuilder(final Configuration configuration, final Logger logger) { this.configuration = Preconditions.checkNotNull(configuration); this.logger = Preconditions.checkNotNull(logger); this.externalAddress = null; @@ -283,24 +283,24 @@ private AkkaRpcServiceBuilder(final Configuration configuration, final Logger lo } @Override - public AkkaRpcServiceBuilder withComponentName(final String actorSystemName) { + public PekkoRpcServiceBuilder withComponentName(final String actorSystemName) { this.actorSystemName = Preconditions.checkNotNull(actorSystemName); return this; } - public AkkaRpcServiceBuilder withCustomConfig(final Config customConfig) { + public PekkoRpcServiceBuilder withCustomConfig(final Config customConfig) { this.customConfig = customConfig; return this; } @Override - public AkkaRpcServiceBuilder withBindAddress(final String bindAddress) { + public PekkoRpcServiceBuilder withBindAddress(final String bindAddress) { this.bindAddress = Preconditions.checkNotNull(bindAddress); return this; } @Override - public AkkaRpcServiceBuilder withBindPort(int bindPort) { + public PekkoRpcServiceBuilder withBindPort(int bindPort) { Preconditions.checkArgument( NetUtils.isValidHostPort(bindPort), "Invalid port number: " + bindPort); this.bindPort = bindPort; @@ -311,7 +311,7 @@ public AkkaRpcServiceBuilder withBindPort(int bindPort) { public RpcSystem.RpcServiceBuilder withExecutorConfiguration( RpcSystem.FixedThreadPoolExecutorConfiguration executorConfiguration) { this.actorSystemExecutorConfiguration = - AkkaUtils.getThreadPoolExecutorConfig(executorConfiguration); + PekkoUtils.getThreadPoolExecutorConfig(executorConfiguration); return this; } @@ -319,34 +319,35 @@ public RpcSystem.RpcServiceBuilder withExecutorConfiguration( public RpcSystem.RpcServiceBuilder withExecutorConfiguration( RpcSystem.ForkJoinExecutorConfiguration executorConfiguration) { this.actorSystemExecutorConfiguration = - AkkaUtils.getForkJoinExecutorConfig(executorConfiguration); + PekkoUtils.getForkJoinExecutorConfig(executorConfiguration); return this; } - public AkkaRpcService createAndStart() throws Exception { - return createAndStart(AkkaRpcService::new); + public PekkoRpcService createAndStart() throws Exception { + return createAndStart(PekkoRpcService::new); } - public AkkaRpcService createAndStart( - TriFunction + public PekkoRpcService createAndStart( + TriFunction constructor) throws Exception { if (actorSystemExecutorConfiguration == null) { actorSystemExecutorConfiguration = - AkkaUtils.getForkJoinExecutorConfig( - AkkaBootstrapTools.getForkJoinExecutorConfiguration(configuration)); + PekkoUtils.getForkJoinExecutorConfig( + ActorSystemBootstrapTools.getForkJoinExecutorConfiguration( + configuration)); } final ActorSystem actorSystem; - // akka internally caches the context class loader + // pekko internally caches the context class loader // make sure it uses the plugin class loader try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(getClass().getClassLoader())) { if (externalAddress == null) { // create local actor system actorSystem = - AkkaBootstrapTools.startLocalActorSystem( + ActorSystemBootstrapTools.startLocalActorSystem( configuration, actorSystemName, logger, @@ -355,7 +356,7 @@ public AkkaRpcService createAndStart( } else { // create remote actor system actorSystem = - AkkaBootstrapTools.startRemoteActorSystem( + ActorSystemBootstrapTools.startRemoteActorSystem( configuration, actorSystemName, externalAddress, @@ -370,7 +371,7 @@ public AkkaRpcService createAndStart( return constructor.apply( actorSystem, - AkkaRpcServiceConfiguration.fromConfiguration(configuration), + PekkoRpcServiceConfiguration.fromConfiguration(configuration), RpcService.class.getClassLoader()); } } @@ -378,5 +379,5 @@ public AkkaRpcService createAndStart( // ------------------------------------------------------------------------ /** This class is not meant to be instantiated. */ - private AkkaRpcServiceUtils() {} + private PekkoRpcServiceUtils() {} } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystem.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystem.java similarity index 81% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystem.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystem.java index 1849d9df287a7..1b5b8cb71360d 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSystem.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcSystem.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.rpc.AddressResolution; @@ -26,12 +26,12 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; -/** {@link RpcSystem} implementation based on Akka. */ -public class AkkaRpcSystem implements RpcSystem { +/** {@link RpcSystem} implementation based on Pekko. */ +public class PekkoRpcSystem implements RpcSystem { @Override public RpcServiceBuilder localServiceBuilder(Configuration configuration) { - return AkkaRpcServiceUtils.localServiceBuilder(configuration); + return PekkoRpcServiceUtils.localServiceBuilder(configuration); } @Override @@ -39,13 +39,13 @@ public RpcServiceBuilder remoteServiceBuilder( Configuration configuration, @Nullable String externalAddress, String externalPortRange) { - return AkkaRpcServiceUtils.remoteServiceBuilder( + return PekkoRpcServiceUtils.remoteServiceBuilder( configuration, externalAddress, externalPortRange); } @Override public InetSocketAddress getInetSocketAddressFromRpcUrl(String url) throws Exception { - return AkkaUtils.getInetSocketAddressFromAkkaURL(url); + return PekkoUtils.getInetSocketAddressFromRpcURL(url); } @Override @@ -56,12 +56,12 @@ public String getRpcUrl( AddressResolution addressResolution, Configuration config) throws UnknownHostException { - return AkkaRpcServiceUtils.getRpcUrl( + return PekkoRpcServiceUtils.getRpcUrl( hostname, port, endpointName, addressResolution, config); } @Override public long getMaximumMessageSizeInBytes(Configuration config) { - return AkkaRpcServiceUtils.extractMaximumFramesize(config); + return PekkoRpcServiceUtils.extractMaximumFramesize(config); } } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaUtils.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoUtils.java similarity index 69% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaUtils.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoUtils.java index 65bf7a7f41bda..12ab0b2d3dc82 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaUtils.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoUtils.java @@ -15,24 +15,24 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.SecurityOptions; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.RpcSystem; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TimeUtils; import org.apache.flink.util.function.FunctionUtils; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Address; -import akka.actor.AddressFromURIString; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Address; +import org.apache.pekko.actor.AddressFromURIString; import org.jboss.netty.logging.InternalLoggerFactory; import org.jboss.netty.logging.Slf4JLoggerFactory; import org.slf4j.Logger; @@ -50,12 +50,12 @@ import java.util.stream.Collectors; /** - * This class contains utility functions for akka. It contains methods to start an actor system with - * a given akka configuration. Furthermore, the akka configuration used for starting the different - * actor systems resides in this class. + * This class contains utility functions for pekko. It contains methods to start an actor system + * with a given Pekko configuration. Furthermore, the Pekko configuration used for starting the + * different actor systems resides in this class. */ -class AkkaUtils { - private static final Logger LOG = LoggerFactory.getLogger(AkkaUtils.class); +class PekkoUtils { + private static final Logger LOG = LoggerFactory.getLogger(PekkoUtils.class); private static final String FLINK_ACTOR_SYSTEM_NAME = "flink"; @@ -64,24 +64,24 @@ public static String getFlinkActorSystemName() { } /** - * Gets the basic Akka config which is shared by remote and local actor systems. + * Gets the basic Pekko config which is shared by remote and local actor systems. * * @param configuration instance which contains the user specified values for the configuration - * @return Flink's basic Akka config + * @return Flink's basic Pekko config */ - private static Config getBasicAkkaConfig(Configuration configuration) { - final int akkaThroughput = configuration.getInteger(AkkaOptions.DISPATCHER_THROUGHPUT); + private static Config getBasicConfig(Configuration configuration) { + final int throughput = configuration.getInteger(AkkaOptions.DISPATCHER_THROUGHPUT); final String jvmExitOnFatalError = booleanToOnOrOff(configuration.getBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR)); final String logLifecycleEvents = booleanToOnOrOff(configuration.getBoolean(AkkaOptions.LOG_LIFECYCLE_EVENTS)); final String supervisorStrategy = EscalatingSupervisorStrategy.class.getCanonicalName(); - return new AkkaConfigBuilder() - .add("akka {") + return new ConfigBuilder() + .add("pekko {") .add(" daemonic = off") - .add(" loggers = [\"akka.event.slf4j.Slf4jLogger\"]") - .add(" logging-filter = \"akka.event.slf4j.Slf4jLoggingFilter\"") + .add(" loggers = [\"org.apache.pekko.event.slf4j.Slf4jLogger\"]") + .add(" logging-filter = \"org.apache.pekko.event.slf4j.Slf4jLoggingFilter\"") .add(" log-config-on-start = off") .add(" logger-startup-timeout = 50s") .add(" loglevel = " + getLogLevel()) @@ -95,7 +95,7 @@ private static Config getBasicAkkaConfig(Configuration configuration) { .add(" warn-about-java-serializer-usage = off") .add(" allow-java-serialization = on") .add(" default-dispatcher {") - .add(" throughput = " + akkaThroughput) + .add(" throughput = " + throughput) .add(" }") .add(" supervisor-dispatcher {") .add(" type = Dispatcher") @@ -112,7 +112,7 @@ private static Config getBasicAkkaConfig(Configuration configuration) { private static String getLogLevel() { if (LOG.isTraceEnabled()) { - // TRACE is not supported by akka + // TRACE is not supported by Pekko return "DEBUG"; } if (LOG.isDebugEnabled()) { @@ -136,11 +136,11 @@ public static Config getThreadPoolExecutorConfig( final int minNumThreads = configuration.getMinNumThreads(); final int maxNumThreads = configuration.getMaxNumThreads(); - return new AkkaConfigBuilder() - .add("akka {") + return new ConfigBuilder() + .add("pekko {") .add(" actor {") .add(" default-dispatcher {") - .add(" type = org.apache.flink.runtime.rpc.akka.PriorityThreadsDispatcher") + .add(" type = " + PriorityThreadsDispatcher.class.getCanonicalName()) .add(" executor = thread-pool-executor") .add(" thread-priority = " + threadPriority) .add(" thread-pool-executor {") @@ -159,8 +159,8 @@ public static Config getForkJoinExecutorConfig( final int minNumThreads = configuration.getMinParallelism(); final int maxNumThreads = configuration.getMaxParallelism(); - return new AkkaConfigBuilder() - .add("akka {") + return new ConfigBuilder() + .add("pekko {") .add(" actor {") .add(" default-dispatcher {") .add(" executor = fork-join-executor") @@ -176,37 +176,34 @@ public static Config getForkJoinExecutorConfig( } /** - * Creates a Akka config for a remote actor system listening on port on the network interface + * Creates a Pekko config for a remote actor system listening on port on the network interface * identified by bindAddress. * * @param configuration instance containing the user provided configuration values * @param bindAddress of the network interface to bind on - * @param port to bind to or if 0 then Akka picks a free port automatically - * @param externalHostname The host name to expect for Akka messages - * @param externalPort The port to expect for Akka messages - * @return Flink's Akka configuration for remote actor systems + * @param port to bind to or if 0 then Pekko picks a free port automatically + * @param externalHostname The host name to expect for Pekko messages + * @param externalPort The port to expect for Pekko messages + * @return Flink's Pekko configuration for remote actor systems */ - private static Config getRemoteAkkaConfig( + private static Config getRemoteConfig( Configuration configuration, String bindAddress, int port, String externalHostname, int externalPort) { - final AkkaConfigBuilder builder = new AkkaConfigBuilder(); + final ConfigBuilder builder = new ConfigBuilder(); - addBaseRemoteAkkaConfig(builder, configuration, port, externalPort); - addHostnameRemoteAkkaConfig(builder, bindAddress, externalHostname); - addSslRemoteAkkaConfig(builder, configuration); + addBaseRemoteConfig(builder, configuration, port, externalPort); + addHostnameRemoteConfig(builder, bindAddress, externalHostname); + addSslRemoteConfig(builder, configuration); return builder.build(); } - private static void addBaseRemoteAkkaConfig( - AkkaConfigBuilder akkaConfigBuilder, - Configuration configuration, - int port, - int externalPort) { - final Duration akkaAskTimeout = configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION); + private static void addBaseRemoteConfig( + ConfigBuilder configBuilder, Configuration configuration, int port, int externalPort) { + final Duration askTimeout = configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION); final String startupTimeout = TimeUtils.getStringInMillis( @@ -214,13 +211,13 @@ private static void addBaseRemoteAkkaConfig( configuration.getString( AkkaOptions.STARTUP_TIMEOUT, TimeUtils.getStringInMillis( - akkaAskTimeout.multipliedBy(10L))))); + askTimeout.multipliedBy(10L))))); - final String akkaTCPTimeout = + final String tcpTimeout = TimeUtils.getStringInMillis( TimeUtils.parseDuration(configuration.getString(AkkaOptions.TCP_TIMEOUT))); - final String akkaFramesize = configuration.getString(AkkaOptions.FRAMESIZE); + final String framesize = configuration.getString(AkkaOptions.FRAMESIZE); final int clientSocketWorkerPoolPoolSizeMin = configuration.get(AkkaOptions.CLIENT_SOCKET_WORKER_POOL_SIZE_MIN); @@ -240,10 +237,10 @@ private static void addBaseRemoteAkkaConfig( final long retryGateClosedFor = configuration.getLong(AkkaOptions.RETRY_GATE_CLOSED_FOR); - akkaConfigBuilder - .add("akka {") + configBuilder + .add("pekko {") .add(" actor {") - .add(" provider = \"akka.remote.RemoteActorRefProvider\"") + .add(" provider = \"org.apache.pekko.remote.RemoteActorRefProvider\"") .add(" }") .add(" remote.artery.enabled = false") .add(" remote.startup-timeout = " + startupTimeout) @@ -256,14 +253,15 @@ private static void addBaseRemoteAkkaConfig( .add(" heartbeat-interval = 1000 s") .add(" threshold = 300") .add(" }") - .add(" enabled-transports = [\"akka.remote.classic.netty.tcp\"]") + .add(" enabled-transports = [\"pekko.remote.classic.netty.tcp\"]") .add(" netty {") .add(" tcp {") - .add(" transport-class = \"akka.remote.transport.netty.NettyTransport\"") + .add( + " transport-class = \"org.apache.pekko.remote.transport.netty.NettyTransport\"") .add(" port = " + externalPort) .add(" bind-port = " + port) - .add(" connection-timeout = " + akkaTCPTimeout) - .add(" maximum-frame-size = " + akkaFramesize) + .add(" connection-timeout = " + tcpTimeout) + .add(" maximum-frame-size = " + framesize) .add(" tcp-nodelay = on") .add(" client-socket-worker-pool {") .add(" pool-size-min = " + clientSocketWorkerPoolPoolSizeMin) @@ -283,19 +281,19 @@ private static void addBaseRemoteAkkaConfig( .add("}"); } - private static void addHostnameRemoteAkkaConfig( - AkkaConfigBuilder akkaConfigBuilder, String bindAddress, String externalHostname) { + private static void addHostnameRemoteConfig( + ConfigBuilder configBuilder, String bindAddress, String externalHostname) { final String normalizedExternalHostname = NetUtils.unresolvedHostToNormalizedString(externalHostname); final String effectiveHostname = normalizedExternalHostname != null && !normalizedExternalHostname.isEmpty() ? normalizedExternalHostname - // if bindAddress is null or empty, then leave bindAddress unspecified. Akka - // will pick InetAddress.getLocalHost.getHostAddress + // if bindAddress is null or empty, then leave bindAddress unspecified. + // Pekko will pick InetAddress.getLocalHost.getHostAddress : ""; - akkaConfigBuilder - .add("akka {") + configBuilder + .add("pekko {") .add(" remote.classic {") .add(" netty {") .add(" tcp {") @@ -307,79 +305,78 @@ private static void addHostnameRemoteAkkaConfig( .add("}"); } - private static void addSslRemoteAkkaConfig( - AkkaConfigBuilder akkaConfigBuilder, Configuration configuration) { + private static void addSslRemoteConfig( + ConfigBuilder configBuilder, Configuration configuration) { - final boolean akkaEnableSSLConfig = + final boolean enableSSLConfig = configuration.getBoolean(AkkaOptions.SSL_ENABLED) && SecurityOptions.isInternalSSLEnabled(configuration); - final String akkaEnableSSL = booleanToOnOrOff(akkaEnableSSLConfig); + final String enableSSL = booleanToOnOrOff(enableSSLConfig); - final String akkaSSLKeyStore = + final String sslKeyStore = configuration.getString( SecurityOptions.SSL_INTERNAL_KEYSTORE, configuration.getString(SecurityOptions.SSL_KEYSTORE)); - final String akkaSSLKeyStorePassword = + final String sslKeyStorePassword = configuration.getString( SecurityOptions.SSL_INTERNAL_KEYSTORE_PASSWORD, configuration.getString(SecurityOptions.SSL_KEYSTORE_PASSWORD)); - final String akkaSSLKeyPassword = + final String sslKeyPassword = configuration.getString( SecurityOptions.SSL_INTERNAL_KEY_PASSWORD, configuration.getString(SecurityOptions.SSL_KEY_PASSWORD)); - final String akkaSSLTrustStore = + final String sslTrustStore = configuration.getString( SecurityOptions.SSL_INTERNAL_TRUSTSTORE, configuration.getString(SecurityOptions.SSL_TRUSTSTORE)); - final String akkaSSLTrustStorePassword = + final String sslTrustStorePassword = configuration.getString( SecurityOptions.SSL_INTERNAL_TRUSTSTORE_PASSWORD, configuration.getString(SecurityOptions.SSL_TRUSTSTORE_PASSWORD)); - final String akkaSSLCertFingerprintString = + final String sslCertFingerprintString = configuration.getString(SecurityOptions.SSL_INTERNAL_CERT_FINGERPRINT); - final String akkaSSLCertFingerprints = - akkaSSLCertFingerprintString != null - ? Arrays.stream(akkaSSLCertFingerprintString.split(",")) + final String sslCertFingerprints = + sslCertFingerprintString != null + ? Arrays.stream(sslCertFingerprintString.split(",")) .collect(Collectors.joining("\",\"", "[\"", "\"]")) : "[]"; - final String akkaSSLProtocol = configuration.getString(SecurityOptions.SSL_PROTOCOL); + final String sslProtocol = configuration.getString(SecurityOptions.SSL_PROTOCOL); - final String akkaSSLAlgorithmsString = - configuration.getString(SecurityOptions.SSL_ALGORITHMS); - final String akkaSSLAlgorithms = - Arrays.stream(akkaSSLAlgorithmsString.split(",")) + final String sslAlgorithmsString = configuration.getString(SecurityOptions.SSL_ALGORITHMS); + final String sslAlgorithms = + Arrays.stream(sslAlgorithmsString.split(",")) .collect(Collectors.joining(",", "[", "]")); final String sslEngineProviderName = CustomSSLEngineProvider.class.getCanonicalName(); - akkaConfigBuilder - .add("akka {") + configBuilder + .add("pekko {") .add(" remote.classic {") - .add(" enabled-transports = [\"akka.remote.classic.netty.ssl\"]") + .add(" enabled-transports = [\"pekko.remote.classic.netty.ssl\"]") .add(" netty {") - .add(" ssl = ${akka.remote.classic.netty.tcp}") + .add(" ssl = ${pekko.remote.classic.netty.tcp}") .add(" ssl {") - .add(" enable-ssl = " + akkaEnableSSL) + .add(" enable-ssl = " + enableSSL) .add(" ssl-engine-provider = " + sslEngineProviderName) .add(" security {") - .add(" key-store = \"" + akkaSSLKeyStore + "\"") - .add(" key-store-password = \"" + akkaSSLKeyStorePassword + "\"") - .add(" key-password = \"" + akkaSSLKeyPassword + "\"") - .add(" trust-store = \"" + akkaSSLTrustStore + "\"") - .add(" trust-store-password = \"" + akkaSSLTrustStorePassword + "\"") - .add(" protocol = " + akkaSSLProtocol + "") - .add(" enabled-algorithms = " + akkaSSLAlgorithms + "") + .add(" key-store = \"" + sslKeyStore + "\"") + .add(" key-store-password = \"" + sslKeyStorePassword + "\"") + .add(" key-password = \"" + sslKeyPassword + "\"") + .add(" trust-store = \"" + sslTrustStore + "\"") + .add(" trust-store-password = \"" + sslTrustStorePassword + "\"") + .add(" protocol = " + sslProtocol + "") + .add(" enabled-algorithms = " + sslAlgorithms + "") .add(" random-number-generator = \"\"") .add(" require-mutual-authentication = on") - .add(" cert-fingerprints = " + akkaSSLCertFingerprints + "") + .add(" cert-fingerprints = " + sslCertFingerprints + "") .add(" }") .add(" }") .add(" }") @@ -394,31 +391,30 @@ private static void addSslRemoteAkkaConfig( * @return The created actor system */ public static ActorSystem createLocalActorSystem(Configuration configuration) { - final Config akkaConfig = getAkkaConfig(configuration, null); - return createActorSystem(akkaConfig); + return createActorSystem(getConfig(configuration, null)); } /** - * Creates an actor system with the given akka config. + * Creates an actor system with the given pekko config. * - * @param akkaConfig configuration for the actor system + * @param config configuration for the actor system * @return created actor system */ - private static ActorSystem createActorSystem(Config akkaConfig) { - return createActorSystem(getFlinkActorSystemName(), akkaConfig); + private static ActorSystem createActorSystem(Config config) { + return createActorSystem(getFlinkActorSystemName(), config); } /** - * Creates an actor system with the given akka config. + * Creates an actor system with the given pekko config. * * @param actorSystemName name of the actor system - * @param akkaConfig configuration for the actor system + * @param config configuration for the actor system * @return created actor system */ - public static ActorSystem createActorSystem(String actorSystemName, Config akkaConfig) { - // Initialize slf4j as logger of Akka's Netty instead of java.util.logging (FLINK-1650) + public static ActorSystem createActorSystem(String actorSystemName, Config config) { + // Initialize slf4j as logger of Pekko's Netty instead of java.util.logging (FLINK-1650) InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()); - return RobustActorSystem.create(actorSystemName, akkaConfig); + return RobustActorSystem.create(actorSystemName, config); } /** @@ -429,64 +425,64 @@ public static ActorSystem createActorSystem(String actorSystemName, Config akkaC */ @VisibleForTesting public static ActorSystem createDefaultActorSystem() { - return createActorSystem(getDefaultAkkaConfig()); + return createActorSystem(getDefaultConfig()); } /** - * Creates the default akka configuration which listens on a random port on the local machine. + * Creates the default pekko configuration which listens on a random port on the local machine. * All configuration values are set to default values. * - * @return Flink's Akka default config + * @return Flink's Pekko default config */ - private static Config getDefaultAkkaConfig() { - return getAkkaConfig(new Configuration(), new HostAndPort("", 0)); + private static Config getDefaultConfig() { + return getConfig(new Configuration(), new HostAndPort("", 0)); } /** - * Creates an akka config with the provided configuration values. If the listening address is + * Creates a pekko config with the provided configuration values. If the listening address is * specified, then the actor system will listen on the respective address. * * @param configuration instance containing the user provided configuration values * @param externalAddress optional tuple of bindAddress and port to be reachable at. If null is - * given, then an Akka config for local actor system will be returned - * @return Akka config + * given, then a Pekko config for local actor system will be returned + * @return Pekko config */ - public static Config getAkkaConfig( + public static Config getConfig( Configuration configuration, @Nullable HostAndPort externalAddress) { - return getAkkaConfig( + return getConfig( configuration, externalAddress, null, - AkkaUtils.getForkJoinExecutorConfig( - AkkaBootstrapTools.getForkJoinExecutorConfiguration(configuration))); + PekkoUtils.getForkJoinExecutorConfig( + ActorSystemBootstrapTools.getForkJoinExecutorConfiguration(configuration))); } /** - * Creates an akka config with the provided configuration values. If the listening address is + * Creates a pekko config with the provided configuration values. If the listening address is * specified, then the actor system will listen on the respective address. * * @param configuration instance containing the user provided configuration values * @param externalAddress optional tuple of external address and port to be reachable at. If - * null is given, then an Akka config for local actor system will be returned + * null is given, then a Pekko config for local actor system will be returned * @param bindAddress optional tuple of bind address and port to be used locally. If null is * given, wildcard IP address and the external port wil be used. Takes effect only if * externalAddress is not null. * @param executorConfig config defining the used executor by the default dispatcher - * @return Akka config + * @return Pekko config */ - public static Config getAkkaConfig( + public static Config getConfig( Configuration configuration, @Nullable HostAndPort externalAddress, @Nullable HostAndPort bindAddress, Config executorConfig) { final Config defaultConfig = - AkkaUtils.getBasicAkkaConfig(configuration).withFallback(executorConfig); + PekkoUtils.getBasicConfig(configuration).withFallback(executorConfig); if (externalAddress != null) { if (bindAddress != null) { final Config remoteConfig = - AkkaUtils.getRemoteAkkaConfig( + PekkoUtils.getRemoteConfig( configuration, bindAddress.getHost(), bindAddress.getPort(), @@ -496,7 +492,7 @@ public static Config getAkkaConfig( return remoteConfig.withFallback(defaultConfig); } else { final Config remoteConfig = - AkkaUtils.getRemoteAkkaConfig( + PekkoUtils.getRemoteConfig( configuration, NetUtils.getWildcardIPAddress(), externalAddress.getPort(), @@ -529,39 +525,38 @@ public static Address getAddress(ActorSystem system) { * @param actor {@link ActorRef} of the actor for which the URL has to be generated * @return String containing the {@link ActorSystem} independent URL of the actor */ - public static String getAkkaURL(ActorSystem system, ActorRef actor) { + public static String getRpcURL(ActorSystem system, ActorRef actor) { final Address address = getAddress(system); return actor.path().toStringWithAddress(address); } /** - * Extracts the {@link Address} from the given akka URL. + * Extracts the {@link Address} from the given pekko URL. * - * @param akkaURL to extract the {@link Address} from - * @throws MalformedURLException if the {@link Address} could not be parsed from the given akka + * @param rpcURL to extract the {@link Address} from + * @throws MalformedURLException if the {@link Address} could not be parsed from the given pekko * URL - * @return Extracted {@link Address} from the given akka URL + * @return Extracted {@link Address} from the given rpc URL */ - @SuppressWarnings("RedundantThrows") // hidden checked exception coming from Akka - public static Address getAddressFromAkkaURL(String akkaURL) throws MalformedURLException { - return AddressFromURIString.apply(akkaURL); + @SuppressWarnings("RedundantThrows") // hidden checked exception coming from Pekko + public static Address getAddressFromRpcURL(String rpcURL) throws MalformedURLException { + return AddressFromURIString.apply(rpcURL); } /** - * Extracts the hostname and the port of the remote actor system from the given Akka URL. The + * Extracts the hostname and the port of the remote actor system from the given Pekko URL. The * result is an {@link InetSocketAddress} instance containing the extracted hostname and port. - * If the Akka URL does not contain the hostname and port information, e.g. a local Akka URL is - * provided, then an {@link Exception} is thrown. + * If the Pekko URL does not contain the hostname and port information, e.g. a local Pekko URL + * is provided, then an {@link Exception} is thrown. * - * @param akkaURL The URL to extract the host and port from. + * @param rpcURL The URL to extract the host and port from. * @throws java.lang.Exception Thrown, if the given string does not represent a proper url * @return The InetSocketAddress with the extracted host and port. */ - public static InetSocketAddress getInetSocketAddressFromAkkaURL(String akkaURL) - throws Exception { - // AkkaURLs have the form schema://systemName@host:port/.... if it's a remote Akka URL + public static InetSocketAddress getInetSocketAddressFromRpcURL(String rpcURL) throws Exception { + // Pekko URLs have the form schema://systemName@host:port/.... if it's a remote Pekko URL try { - final Address address = getAddressFromAkkaURL(akkaURL); + final Address address = getAddressFromRpcURL(rpcURL); if (address.host().isDefined() && address.port().isDefined()) { return new InetSocketAddress(address.host().get(), (int) address.port().get()); @@ -569,7 +564,7 @@ public static InetSocketAddress getInetSocketAddressFromAkkaURL(String akkaURL) throw new MalformedURLException(); } } catch (MalformedURLException e) { - throw new Exception("Could not retrieve InetSocketAddress from Akka URL " + akkaURL); + throw new Exception("Could not retrieve InetSocketAddress from Pekko URL " + rpcURL); } } @@ -580,18 +575,19 @@ public static InetSocketAddress getInetSocketAddressFromAkkaURL(String akkaURL) * @return Termination future */ public static CompletableFuture terminateActorSystem(ActorSystem actorSystem) { - return AkkaFutureUtils.toJava(actorSystem.terminate()).thenAccept(FunctionUtils.ignoreFn()); + return ScalaFutureUtils.toJava(actorSystem.terminate()) + .thenAccept(FunctionUtils.ignoreFn()); } private static String booleanToOnOrOff(boolean flag) { return flag ? "on" : "off"; } - private static class AkkaConfigBuilder { + private static class ConfigBuilder { private final StringWriter stringWriter = new StringWriter(); private final PrintWriter printWriter = new PrintWriter(stringWriter); - public AkkaConfigBuilder add(String configLine) { + public ConfigBuilder add(String configLine) { printWriter.println(configLine); return this; } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PrioritySettingThreadFactory.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PrioritySettingThreadFactory.java similarity index 97% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PrioritySettingThreadFactory.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PrioritySettingThreadFactory.java index c4ee3b0706294..34cf9c1800104 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PrioritySettingThreadFactory.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PrioritySettingThreadFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import javax.annotation.Nonnull; diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PriorityThreadsDispatcher.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PriorityThreadsDispatcher.java similarity index 77% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PriorityThreadsDispatcher.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PriorityThreadsDispatcher.java index 47504eb9e94e4..aaf26d56f2fa8 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/PriorityThreadsDispatcher.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PriorityThreadsDispatcher.java @@ -15,22 +15,22 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; -import akka.dispatch.DefaultDispatcherPrerequisites; -import akka.dispatch.DispatcherConfigurator; -import akka.dispatch.DispatcherPrerequisites; import com.typesafe.config.Config; +import org.apache.pekko.dispatch.DefaultDispatcherPrerequisites; +import org.apache.pekko.dispatch.DispatcherConfigurator; +import org.apache.pekko.dispatch.DispatcherPrerequisites; /** - * Akka dispatcher threads creates threads with configurable priority. + * Pekko dispatcher threads creates threads with configurable priority. * *

Example of configuration: * *

  *
  *   low-priority-threads-dispatcher {
- *     type = org.apache.flink.runtime.rpc.akka.PriorityThreadsDispatcher
+ *     type = org.apache.flink.runtime.rpc.pekko.PriorityThreadsDispatcher
  *     executor = "thread-pool-executor"
  *     # should be between Thread.MIN_PRIORITY (which is 1) and Thread.MAX_PRIORITY (which is 10)
  *     threads-priority = 1
@@ -42,20 +42,21 @@
  *   }
  * 
* - *

Two arguments constructor (the primary constructor) is automatically called by Akka when it + *

Two arguments constructor (the primary constructor) is automatically called by Pekko when it * finds: * *

  *   abcde-dispatcher {
- *     type = org.apache.flink.runtime.rpc.akka.PriorityThreadsDispatcher <-- the class that Akka will instantiate
+ *     type = org.apache.flink.runtime.rpc.pekko.PriorityThreadsDispatcher <-- the class that Pekko will instantiate
  *     ...
  *   }
  * 
*/ public class PriorityThreadsDispatcher extends DispatcherConfigurator { /** - * @param config passed automatically by Akka, should contain information about threads priority - * @param prerequisites passed automatically by Akka + * @param config passed automatically by Pekko, should contain information about threads + * priority + * @param prerequisites passed automatically by Pekko */ public PriorityThreadsDispatcher(Config config, DispatcherPrerequisites prerequisites) { super( diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RemoteAddressExtension.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RemoteAddressExtension.java similarity index 81% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RemoteAddressExtension.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RemoteAddressExtension.java index ff02d72d70c25..aaab82e17d98d 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RemoteAddressExtension.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RemoteAddressExtension.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; -import akka.actor.AbstractExtensionId; -import akka.actor.Address; -import akka.actor.ExtendedActorSystem; -import akka.actor.Extension; +import org.apache.pekko.actor.AbstractExtensionId; +import org.apache.pekko.actor.Address; +import org.apache.pekko.actor.ExtendedActorSystem; +import org.apache.pekko.actor.Extension; /** - * {@link akka.actor.ActorSystem} {@link Extension} used to obtain the {@link Address} on which the - * given ActorSystem is listening. + * {@link org.apache.pekko.actor.ActorSystem} {@link Extension} used to obtain the {@link Address} + * on which the given ActorSystem is listening. */ public class RemoteAddressExtension extends AbstractExtensionId { diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RobustActorSystem.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystem.java similarity index 95% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RobustActorSystem.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystem.java index c4a547127d343..031851b00cb23 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/RobustActorSystem.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystem.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.FatalExitExceptionHandler; -import akka.actor.ActorSystemImpl; -import akka.actor.BootstrapSetup; -import akka.actor.setup.ActorSystemSetup; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; +import org.apache.pekko.actor.ActorSystemImpl; +import org.apache.pekko.actor.BootstrapSetup; +import org.apache.pekko.actor.setup.ActorSystemSetup; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; @@ -123,7 +123,7 @@ public void uncaughtException(Thread t, Throwable e) { if (shutdownComplete.get() && (e instanceof NoClassDefFoundError || e instanceof ClassNotFoundException)) { // ignore classloading errors after the actor system terminated - // some parts of the akka shutdown procedure are not tied to the actor + // some parts of the pekko shutdown procedure are not tied to the actor // system termination future, and can occasionally fail if the rpc // classloader has been closed. return; diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValue.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValue.java similarity index 82% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValue.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValue.java index fb8011cf5c677..a0ec892c74a5e 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValue.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValue.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; @@ -28,12 +28,12 @@ import java.util.Arrays; /** A self-contained serialized value to decouple from user values and transfer on wire. */ -final class AkkaRpcSerializedValue implements Serializable { +final class RpcSerializedValue implements Serializable { private static final long serialVersionUID = -4388571068440835689L; @Nullable private final byte[] serializedData; - private AkkaRpcSerializedValue(@Nullable byte[] serializedData) { + private RpcSerializedValue(@Nullable byte[] serializedData) { this.serializedData = serializedData; } @@ -62,15 +62,15 @@ public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFo * @return serialized value to transfer on wire * @throws IOException exception during value serialization */ - public static AkkaRpcSerializedValue valueOf(@Nullable Object value) throws IOException { + public static RpcSerializedValue valueOf(@Nullable Object value) throws IOException { byte[] serializedData = value == null ? null : InstantiationUtil.serializeObject(value); - return new AkkaRpcSerializedValue(serializedData); + return new RpcSerializedValue(serializedData); } @Override public boolean equals(Object o) { - if (o instanceof AkkaRpcSerializedValue) { - AkkaRpcSerializedValue other = (AkkaRpcSerializedValue) o; + if (o instanceof RpcSerializedValue) { + RpcSerializedValue other = (RpcSerializedValue) o; return Arrays.equals(serializedData, other.serializedData); } return false; @@ -83,6 +83,6 @@ public int hashCode() { @Override public String toString() { - return serializedData == null ? "AkkaRpcSerializedValue(null)" : "AkkaRpcSerializedValue"; + return serializedData == null ? "RpcSerializedValue(null)" : "RpcSerializedValue"; } } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/SupervisorActor.java similarity index 65% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/SupervisorActor.java index a4b439e2ab717..870ca9cc33f43 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/SupervisorActor.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/SupervisorActor.java @@ -16,23 +16,24 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; +import org.apache.flink.runtime.rpc.exceptions.RpcException; +import org.apache.flink.runtime.rpc.pekko.exceptions.UnknownMessageException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.FutureUtils; -import akka.AkkaException; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.ChildRestartStats; -import akka.actor.Props; -import akka.actor.SupervisorStrategy; -import akka.japi.pf.DeciderBuilder; -import akka.pattern.Patterns; +import org.apache.pekko.PekkoException; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.ChildRestartStats; +import org.apache.pekko.actor.Props; +import org.apache.pekko.actor.Status; +import org.apache.pekko.actor.SupervisorStrategy; +import org.apache.pekko.japi.pf.DeciderBuilder; +import org.apache.pekko.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ import scala.collection.Iterable; /** - * Supervisor actor which is responsible for starting {@link AkkaRpcActor} instances and monitoring + * Supervisor actor which is responsible for starting {@link PekkoRpcActor} instances and monitoring * when the actors have terminated. */ class SupervisorActor extends AbstractActor { @@ -59,17 +60,17 @@ class SupervisorActor extends AbstractActor { private final Executor terminationFutureExecutor; - private final Map registeredAkkaRpcActors; + private final Map registeredRpcActors; SupervisorActor(Executor terminationFutureExecutor) { this.terminationFutureExecutor = terminationFutureExecutor; - this.registeredAkkaRpcActors = new HashMap<>(); + this.registeredRpcActors = new HashMap<>(); } @Override public Receive createReceive() { return receiveBuilder() - .match(StartAkkaRpcActor.class, this::createStartAkkaRpcActorMessage) + .match(StartRpcActor.class, this::createStartRpcActorMessage) .matchAny(this::handleUnknownMessage) .build(); } @@ -80,11 +81,11 @@ public void postStop() throws Exception { super.postStop(); - for (AkkaRpcActorRegistration actorRegistration : registeredAkkaRpcActors.values()) { - terminateAkkaRpcActorOnStop(actorRegistration); + for (RpcActorRegistration actorRegistration : registeredRpcActors.values()) { + terminateRpcActorOnStop(actorRegistration); } - registeredAkkaRpcActors.clear(); + registeredRpcActors.clear(); } @Override @@ -92,84 +93,81 @@ public SupervisorActorSupervisorStrategy supervisorStrategy() { return new SupervisorActorSupervisorStrategy(); } - private void terminateAkkaRpcActorOnStop(AkkaRpcActorRegistration akkaRpcActorRegistration) { - akkaRpcActorRegistration.terminateExceptionally( - new AkkaRpcException( + private void terminateRpcActorOnStop(RpcActorRegistration rpcActorRegistration) { + rpcActorRegistration.terminateExceptionally( + new RpcException( String.format( "Unexpected closing of %s with name %s.", - getClass().getSimpleName(), - akkaRpcActorRegistration.getEndpointId())), + getClass().getSimpleName(), rpcActorRegistration.getEndpointId())), terminationFutureExecutor); } - private void createStartAkkaRpcActorMessage(StartAkkaRpcActor startAkkaRpcActor) { - final String endpointId = startAkkaRpcActor.getEndpointId(); - final AkkaRpcActorRegistration akkaRpcActorRegistration = - new AkkaRpcActorRegistration(endpointId); + private void createStartRpcActorMessage(StartRpcActor startRpcActor) { + final String endpointId = startRpcActor.getEndpointId(); + final RpcActorRegistration rpcActorRegistration = new RpcActorRegistration(endpointId); - final Props akkaRpcActorProps = - startAkkaRpcActor + final Props rpcActorProps = + startRpcActor .getPropsFactory() - .create(akkaRpcActorRegistration.getInternalTerminationFuture()); + .create(rpcActorRegistration.getInternalTerminationFuture()); LOG.debug( "Starting {} with name {}.", - akkaRpcActorProps.actorClass().getSimpleName(), + rpcActorProps.actorClass().getSimpleName(), endpointId); try { - final ActorRef actorRef = getContext().actorOf(akkaRpcActorProps, endpointId); + final ActorRef actorRef = getContext().actorOf(rpcActorProps, endpointId); - registeredAkkaRpcActors.put(actorRef, akkaRpcActorRegistration); + registeredRpcActors.put(actorRef, rpcActorRegistration); getSender() .tell( - StartAkkaRpcActorResponse.success( + StartRpcActorResponse.success( ActorRegistration.create( actorRef, - akkaRpcActorRegistration - .getExternalTerminationFuture())), + rpcActorRegistration.getExternalTerminationFuture())), getSelf()); - } catch (AkkaException akkaException) { - getSender().tell(StartAkkaRpcActorResponse.failure(akkaException), getSelf()); + } catch (PekkoException e) { + getSender().tell(StartRpcActorResponse.failure(e), getSelf()); } } - private void akkaRpcActorTerminated(ActorRef actorRef) { - final AkkaRpcActorRegistration actorRegistration = removeAkkaRpcActor(actorRef); + private void rpcActorTerminated(ActorRef actorRef) { + final RpcActorRegistration actorRegistration = removeAkkaRpcActor(actorRef); - LOG.debug("AkkaRpcActor {} has terminated.", actorRef.path()); + LOG.debug("RpcActor {} has terminated.", actorRef.path()); actorRegistration.terminate(terminationFutureExecutor); } - private void akkaRpcActorFailed(ActorRef actorRef, Throwable cause) { - LOG.warn("AkkaRpcActor {} has failed. Shutting it down now.", actorRef.path(), cause); + private void rpcActorFailed(ActorRef actorRef, Throwable cause) { + LOG.warn("RpcActor {} has failed. Shutting it down now.", actorRef.path(), cause); - for (Map.Entry registeredAkkaRpcActor : - registeredAkkaRpcActors.entrySet()) { - final ActorRef otherActorRef = registeredAkkaRpcActor.getKey(); + for (Map.Entry registeredRpcActor : + registeredRpcActors.entrySet()) { + final ActorRef otherActorRef = registeredRpcActor.getKey(); if (otherActorRef.equals(actorRef)) { - final AkkaRpcException error = - new AkkaRpcException( + final RpcException error = + new RpcException( String.format( "Stopping actor %s because it failed.", actorRef.path()), cause); - registeredAkkaRpcActor.getValue().markFailed(error); + registeredRpcActor.getValue().markFailed(error); } else { - final AkkaRpcException siblingException = - new AkkaRpcException( + final RpcException siblingException = + new RpcException( String.format( "Stopping actor %s because its sibling %s has failed.", otherActorRef.path(), actorRef.path())); - registeredAkkaRpcActor.getValue().markFailed(siblingException); + registeredRpcActor.getValue().markFailed(siblingException); } } getContext().getSystem().terminate(); } - private AkkaRpcActorRegistration removeAkkaRpcActor(ActorRef actorRef) { - return Optional.ofNullable(registeredAkkaRpcActors.remove(actorRef)) + private RpcActorRegistration removeAkkaRpcActor(ActorRef actorRef) { + return Optional.ofNullable(registeredRpcActors.remove(actorRef)) .orElseThrow( () -> new IllegalStateException( @@ -178,39 +176,39 @@ private AkkaRpcActorRegistration removeAkkaRpcActor(ActorRef actorRef) { } private void handleUnknownMessage(Object msg) { - final AkkaUnknownMessageException cause = - new AkkaUnknownMessageException( + final UnknownMessageException cause = + new UnknownMessageException( String.format("Cannot handle unknown message %s.", msg)); - getSender().tell(new akka.actor.Status.Failure(cause), getSelf()); + getSender().tell(new Status.Failure(cause), getSelf()); throw cause; } public static String getActorName() { - return AkkaRpcServiceUtils.SUPERVISOR_NAME; + return PekkoRpcServiceUtils.SUPERVISOR_NAME; } public static ActorRef startSupervisorActor( ActorSystem actorSystem, Executor terminationFutureExecutor) { final Props supervisorProps = Props.create(SupervisorActor.class, terminationFutureExecutor) - .withDispatcher("akka.actor.supervisor-dispatcher"); + .withDispatcher("pekko.actor.supervisor-dispatcher"); return actorSystem.actorOf(supervisorProps, getActorName()); } - public static StartAkkaRpcActorResponse startAkkaRpcActor( - ActorRef supervisor, StartAkkaRpcActor.PropsFactory propsFactory, String endpointId) { + public static StartRpcActorResponse startRpcActor( + ActorRef supervisor, StartRpcActor.PropsFactory propsFactory, String endpointId) { return Patterns.ask( supervisor, - createStartAkkaRpcActorMessage(propsFactory, endpointId), + createStartRpcActorMessage(propsFactory, endpointId), RpcUtils.INF_DURATION) .toCompletableFuture() - .thenApply(SupervisorActor.StartAkkaRpcActorResponse.class::cast) + .thenApply(StartRpcActorResponse.class::cast) .join(); } - public static StartAkkaRpcActor createStartAkkaRpcActorMessage( - StartAkkaRpcActor.PropsFactory propsFactory, String endpointId) { - return StartAkkaRpcActor.create(propsFactory, endpointId); + public static StartRpcActor createStartRpcActorMessage( + StartRpcActor.PropsFactory propsFactory, String endpointId) { + return StartRpcActor.create(propsFactory, endpointId); } // ----------------------------------------------------------------------------- @@ -231,13 +229,15 @@ public boolean loggingEnabled() { @Override public void handleChildTerminated( - akka.actor.ActorContext context, ActorRef child, Iterable children) { - akkaRpcActorTerminated(child); + org.apache.pekko.actor.ActorContext context, + ActorRef child, + Iterable children) { + rpcActorTerminated(child); } @Override public void processFailure( - akka.actor.ActorContext context, + org.apache.pekko.actor.ActorContext context, boolean restart, ActorRef child, Throwable cause, @@ -246,11 +246,11 @@ public void processFailure( Preconditions.checkArgument( !restart, "The supervisor strategy should never restart an actor."); - akkaRpcActorFailed(child, cause); + rpcActorFailed(child, cause); } } - private static final class AkkaRpcActorRegistration { + private static final class RpcActorRegistration { private final String endpointId; private final CompletableFuture internalTerminationFuture; @@ -259,7 +259,7 @@ private static final class AkkaRpcActorRegistration { @Nullable private Throwable errorCause; - private AkkaRpcActorRegistration(String endpointId) { + private RpcActorRegistration(String endpointId) { this.endpointId = endpointId; internalTerminationFuture = new CompletableFuture<>(); externalTerminationFuture = new CompletableFuture<>(); @@ -296,7 +296,7 @@ private void terminate(Executor terminationFutureExecutor) { } } else { internalTerminationFuture.completeExceptionally( - new AkkaRpcException( + new RpcException( String.format( "RpcEndpoint %s did not complete the internal termination future.", endpointId))); @@ -324,11 +324,11 @@ public void markFailed(Throwable cause) { // Messages // ----------------------------------------------------------------------------- - static final class StartAkkaRpcActor { + static final class StartRpcActor { private final PropsFactory propsFactory; private final String endpointId; - private StartAkkaRpcActor(PropsFactory propsFactory, String endpointId) { + private StartRpcActor(PropsFactory propsFactory, String endpointId) { this.propsFactory = propsFactory; this.endpointId = endpointId; } @@ -341,8 +341,8 @@ public PropsFactory getPropsFactory() { return propsFactory; } - private static StartAkkaRpcActor create(PropsFactory propsFactory, String endpointId) { - return new StartAkkaRpcActor(propsFactory, endpointId); + private static StartRpcActor create(PropsFactory propsFactory, String endpointId) { + return new StartRpcActor(propsFactory, endpointId); } interface PropsFactory { @@ -373,12 +373,12 @@ public static ActorRegistration create( } } - static final class StartAkkaRpcActorResponse { + static final class StartRpcActorResponse { @Nullable private final ActorRegistration actorRegistration; @Nullable private final Throwable error; - private StartAkkaRpcActorResponse( + private StartRpcActorResponse( @Nullable ActorRegistration actorRegistration, @Nullable Throwable error) { this.actorRegistration = actorRegistration; this.error = error; @@ -393,12 +393,12 @@ public ActorRegistration orElseThrow( } } - public static StartAkkaRpcActorResponse success(ActorRegistration actorRegistration) { - return new StartAkkaRpcActorResponse(actorRegistration, null); + public static StartRpcActorResponse success(ActorRegistration actorRegistration) { + return new StartRpcActorResponse(actorRegistration, null); } - public static StartAkkaRpcActorResponse failure(Throwable error) { - return new StartAkkaRpcActorResponse(null, error); + public static StartRpcActorResponse failure(Throwable error) { + return new StartRpcActorResponse(null, error); } } } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcInvalidStateException.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/RpcInvalidStateException.java similarity index 77% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcInvalidStateException.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/RpcInvalidStateException.java index 1aacd924b11f2..3aa8328800dbe 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcInvalidStateException.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/RpcInvalidStateException.java @@ -16,23 +16,23 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.exceptions; +package org.apache.flink.runtime.rpc.pekko.exceptions; import org.apache.flink.util.FlinkRuntimeException; /** Exception which indicates an invalid state. */ -public class AkkaRpcInvalidStateException extends FlinkRuntimeException { +public class RpcInvalidStateException extends FlinkRuntimeException { private static final long serialVersionUID = 7771345513700817242L; - public AkkaRpcInvalidStateException(String message) { + public RpcInvalidStateException(String message) { super(message); } - public AkkaRpcInvalidStateException(Throwable cause) { + public RpcInvalidStateException(Throwable cause) { super(cause); } - public AkkaRpcInvalidStateException(String message, Throwable cause) { + public RpcInvalidStateException(String message, Throwable cause) { super(message, cause); } } diff --git a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/UnknownMessageException.java similarity index 65% rename from flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java rename to flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/UnknownMessageException.java index b0c582174c695..b2c14ce7285f9 100644 --- a/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java +++ b/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/exceptions/UnknownMessageException.java @@ -16,22 +16,27 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.exceptions; +package org.apache.flink.runtime.rpc.pekko.exceptions; -/** Exception which indicates that the AkkaRpcActor has received an unknown message type. */ -public class AkkaUnknownMessageException extends AkkaRpcRuntimeException { +import org.apache.flink.runtime.rpc.exceptions.RpcRuntimeException; + +/** + * Exception which indicates that the {@link org.apache.flink.runtime.rpc.pekko.PekkoRpcActor} has + * received an unknown message type. + */ +public class UnknownMessageException extends RpcRuntimeException { private static final long serialVersionUID = 1691338049911020814L; - public AkkaUnknownMessageException(String message) { + public UnknownMessageException(String message) { super(message); } - public AkkaUnknownMessageException(String message, Throwable cause) { + public UnknownMessageException(String message, Throwable cause) { super(message, cause); } - public AkkaUnknownMessageException(Throwable cause) { + public UnknownMessageException(Throwable cause) { super(cause); } } diff --git a/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/NOTICE b/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/NOTICE index fb6e3c66c2cd0..caed2e9d4eba3 100644 --- a/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/NOTICE +++ b/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/NOTICE @@ -8,25 +8,24 @@ This project bundles the following dependencies under the Apache Software Licens - com.hierynomus:asn-one:0.5.0 - com.typesafe:config:1.4.2 -- com.typesafe:ssl-config-core_2.12:0.4.3 -- com.typesafe.akka:akka-actor_2.12:2.6.20 -- com.typesafe.akka:akka-remote_2.12:2.6.20 -- com.typesafe.akka:akka-pki_2.12:2.6.20 -- com.typesafe.akka:akka-protobuf-v3_2.12:2.6.20 -- com.typesafe.akka:akka-slf4j_2.12:2.6.20 -- com.typesafe.akka:akka-stream_2.12:2.6.20 +- com.typesafe:ssl-config-core_2.12:0.6.1 - io.netty:netty:3.10.6.Final - org.agrona:agrona:1.15.1 +- org.apache.pekko:pekko-actor_2.12:1.0.0 +- org.apache.pekko:pekko-remote_2.12:1.0.0 +- org.apache.pekko:pekko-pki_2.12:1.0.0 +- org.apache.pekko:pekko-protobuf-v3_2.12:1.0.0 +- org.apache.pekko:pekko-slf4j_2.12:1.0.0 +- org.apache.pekko:pekko-stream_2.12:1.0.0 - org.scala-lang:scala-library:2.12.16 -- org.scala-lang.modules:scala-parser-combinators_2.12:1.1.2 The following dependencies all share the same BSD license which you find under licenses/LICENSE.scala. -- org.scala-lang.modules:scala-java8-compat_2.12:0.8.0 +- org.scala-lang.modules:scala-java8-compat_2.12:1.0.2 This project bundles the following dependencies under the Creative Commons CC0 "No Rights Reserved". -- org.reactivestreams:reactive-streams:1.0.3 +- org.reactivestreams:reactive-streams:1.0.4 This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices: diff --git a/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystem b/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystem index 096f6688e3cd1..8976c9ba6cdba 100644 --- a/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystem +++ b/flink-rpc/flink-rpc-akka/src/main/resources/META-INF/services/org.apache.flink.runtime.rpc.RpcSystem @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.runtime.rpc.akka.AkkaRpcSystem +org.apache.flink.runtime.rpc.pekko.PekkoRpcSystem diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapToolsTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java similarity index 90% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapToolsTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java index b4a3a42580d60..5343c4011cf1c 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaBootstrapToolsTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemBootstrapToolsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.FlinkAssertions; @@ -23,7 +23,7 @@ import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.CheckedSupplier; -import akka.actor.ActorSystem; +import org.apache.pekko.actor.ActorSystem; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,10 +42,10 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the {@link AkkaBootstrapTools}. */ -class AkkaBootstrapToolsTest { +/** Tests for the {@link ActorSystemBootstrapTools}. */ +class ActorSystemBootstrapToolsTest { - private static final Logger LOG = LoggerFactory.getLogger(AkkaBootstrapToolsTest.class); + private static final Logger LOG = LoggerFactory.getLogger(ActorSystemBootstrapToolsTest.class); /** * Tests that we can concurrently create two {@link ActorSystem} without port conflicts. This @@ -68,7 +68,7 @@ void testConcurrentActorSystemCreation() throws Exception { () -> { cyclicBarrier.await(); - return AkkaBootstrapTools + return ActorSystemBootstrapTools .startRemoteActorSystem( new Configuration(), "localhost", @@ -80,7 +80,7 @@ void testConcurrentActorSystemCreation() throws Exception { // terminate ActorSystems actorSystemFuture -> actorSystemFuture.thenCompose( - AkkaUtils::terminateActorSystem)) + PekkoUtils::terminateActorSystem)) .collect(Collectors.toList()); FutureUtils.completeAll(actorSystemFutures).get(); @@ -101,7 +101,7 @@ void testActorSystemInstantiationFailureWhenPortOccupied() throws Exception { final int port = portOccupier.getLocalPort(); assertThatThrownBy( () -> - AkkaBootstrapTools.startRemoteActorSystem( + ActorSystemBootstrapTools.startRemoteActorSystem( new Configuration(), "0.0.0.0", String.valueOf(port), diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ActorSystemExtension.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemExtension.java similarity index 88% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ActorSystemExtension.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemExtension.java index f719504035ae7..f9806554cb1ab 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ActorSystemExtension.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ActorSystemExtension.java @@ -16,17 +16,17 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Preconditions; -import akka.actor.ActorSystem; +import org.apache.pekko.actor.ActorSystem; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeEachCallback; import org.junit.jupiter.api.extension.ExtensionContext; -/** External resource which starts an {@link akka.actor.ActorSystem}. */ +/** External resource which starts an {@link ActorSystem}. */ public class ActorSystemExtension implements BeforeEachCallback, AfterEachCallback { private final Configuration configuration; @@ -41,14 +41,14 @@ private ActorSystemExtension(Configuration configuration) { public void beforeEach(ExtensionContext context) throws Exception { Preconditions.checkState( actorSystem == null, "ActorSystem must not be initialized when calling before."); - actorSystem = AkkaUtils.createLocalActorSystem(configuration); + actorSystem = PekkoUtils.createLocalActorSystem(configuration); } @Override public void afterEach(ExtensionContext context) throws Exception { Preconditions.checkState( actorSystem != null, "ActorSystem must be initialized when calling after."); - AkkaUtils.terminateActorSystem(actorSystem).join(); + PekkoUtils.terminateActorSystem(actorSystem).join(); } public ActorSystem getActorSystem() { diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ContextClassLoadingSettingTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ContextClassLoadingSettingTest.java similarity index 83% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ContextClassLoadingSettingTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ContextClassLoadingSettingTest.java index 911726de4abd8..b18af36b29964 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/ContextClassLoadingSettingTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/ContextClassLoadingSettingTest.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.Local; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -26,8 +26,8 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.concurrent.FutureUtils; -import akka.actor.ActorSystem; -import akka.actor.Terminated; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Terminated; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -53,10 +53,10 @@ import static org.junit.jupiter.api.Assertions.assertEquals; /** - * Tests the context class loader handling in various parts of the akka rpc system. + * Tests the context class loader handling in various parts of the Pekko rpc system. * - *

The tests check cases where we call from the akka rpc system into Flink, in which case the - * context class loader must be set to the Flink class loader. This ensures that the Akka class + *

The tests check cases where we call from the Pekko rpc system into Flink, in which case the + * context class loader must be set to the Flink class loader. This ensures that the Pekko class * loader does not get accidentally leaked, e.g., via thread locals or thread pools on the Flink * side. */ @@ -74,18 +74,18 @@ class ContextClassLoadingSettingTest { private ClassLoader pretendFlinkClassLoader; private ActorSystem actorSystem; - private AkkaRpcService akkaRpcService; + private PekkoRpcService pekkoRpcService; @BeforeEach void setup() { pretendFlinkClassLoader = new URLClassLoader( new URL[0], ContextClassLoadingSettingTest.class.getClassLoader()); - actorSystem = AkkaUtils.createDefaultActorSystem(); - akkaRpcService = - new AkkaRpcService( + actorSystem = PekkoUtils.createDefaultActorSystem(); + pekkoRpcService = + new PekkoRpcService( actorSystem, - AkkaRpcServiceConfiguration.defaultConfiguration(), + PekkoRpcServiceConfiguration.defaultConfiguration(), pretendFlinkClassLoader); PickyObject.classLoaderAssertion = this::assertIsFlinkClassLoader; @@ -93,22 +93,22 @@ void setup() { @AfterEach void shutdown() throws InterruptedException, ExecutionException { - final CompletableFuture rpcTerminationFuture = akkaRpcService.closeAsync(); + final CompletableFuture rpcTerminationFuture = pekkoRpcService.closeAsync(); final CompletableFuture actorSystemTerminationFuture = - AkkaFutureUtils.toJava(actorSystem.terminate()); + ScalaFutureUtils.toJava(actorSystem.terminate()); FutureUtils.waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) .get(); actorSystem = null; - akkaRpcService = null; + pekkoRpcService = null; } @Test - void testAkkaRpcService_ExecuteRunnableSetsFlinkContextClassLoader() + void testRpcService_ExecuteRunnableSetsFlinkContextClassLoader() throws ExecutionException, InterruptedException { final CompletableFuture contextClassLoader = new CompletableFuture<>(); - akkaRpcService + pekkoRpcService .getScheduledExecutor() .execute( () -> @@ -118,10 +118,10 @@ void testAkkaRpcService_ExecuteRunnableSetsFlinkContextClassLoader() } @Test - void testAkkaRpcService_ScheduleCallableSetsFlinkContextClassLoader() + void testRpcService_ScheduleCallableSetsFlinkContextClassLoader() throws ExecutionException, InterruptedException { final ClassLoader contextClassLoader = - akkaRpcService + pekkoRpcService .getScheduledExecutor() .schedule( () -> Thread.currentThread().getContextClassLoader(), @@ -132,10 +132,10 @@ void testAkkaRpcService_ScheduleCallableSetsFlinkContextClassLoader() } @Test - void testAkkaRpcService_ScheduleRunnableSetsFlinkContextClassLoader() + void testRpcService_ScheduleRunnableSetsFlinkContextClassLoader() throws ExecutionException, InterruptedException { final CompletableFuture contextClassLoader = new CompletableFuture<>(); - akkaRpcService + pekkoRpcService .getScheduledExecutor() .schedule( () -> @@ -147,13 +147,13 @@ void testAkkaRpcService_ScheduleRunnableSetsFlinkContextClassLoader() } @Test - void testAkkaRpcService_ScheduleRunnableWithFixedRateSetsFlinkContextClassLoader() + void testRpcService_ScheduleRunnableWithFixedRateSetsFlinkContextClassLoader() throws ExecutionException, InterruptedException { final int numberOfScheduledRuns = 2; final List contextClassLoaders = new ArrayList<>(numberOfScheduledRuns); final CompletableFuture terminalFuture = new CompletableFuture<>(); - akkaRpcService + pekkoRpcService .getScheduledExecutor() .scheduleAtFixedRate( () -> { @@ -176,13 +176,13 @@ void testAkkaRpcService_ScheduleRunnableWithFixedRateSetsFlinkContextClassLoader } @Test - void testAkkaRpcService_ScheduleRunnableWithFixedDelaySetsFlinkContextClassLoader() + void testRpcService_ScheduleRunnableWithFixedDelaySetsFlinkContextClassLoader() throws ExecutionException, InterruptedException { final int numberOfScheduledRuns = 2; final List contextClassLoaders = new ArrayList<>(numberOfScheduledRuns); final CompletableFuture terminalFuture = new CompletableFuture<>(); - akkaRpcService + pekkoRpcService .getScheduledExecutor() .scheduleWithFixedDelay( () -> { @@ -205,13 +205,13 @@ void testAkkaRpcService_ScheduleRunnableWithFixedDelaySetsFlinkContextClassLoade } @Test - void testAkkaRpcService_ConnectFutureCompletedWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcService_ConnectFutureCompletedWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { final ClassLoader contextClassLoader = runWithContextClassLoader( () -> - akkaRpcService + pekkoRpcService .connect( testEndpoint.getAddress(), TestEndpointGateway.class) @@ -226,12 +226,11 @@ void testAkkaRpcService_ConnectFutureCompletedWithFlinkContextClassLoader() thro } @Test - void testAkkaRpcService_TerminationFutureCompletedWithFlinkContextClassLoader() - throws Exception { + void testRpcService_TerminationFutureCompletedWithFlinkContextClassLoader() throws Exception { final ClassLoader contextClassLoader = runWithContextClassLoader( () -> - akkaRpcService + pekkoRpcService .closeAsync() .thenApply( ignored -> @@ -244,16 +243,16 @@ void testAkkaRpcService_TerminationFutureCompletedWithFlinkContextClassLoader() } @Test - void testAkkaRpcActor_OnStartCalledWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcActor_OnStartCalledWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); assertIsFlinkClassLoader(testEndpoint.onStartClassLoader.get()); } } @Test - void testAkkaRpcActor_OnStopCalledWithFlinkContextClassLoader() throws Exception { - final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); + void testRpcActor_OnStopCalledWithFlinkContextClassLoader() throws Exception { + final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService); testEndpoint.start(); testEndpoint.close(); @@ -261,8 +260,8 @@ void testAkkaRpcActor_OnStopCalledWithFlinkContextClassLoader() throws Exception } @Test - void testAkkaRpcActor_CallAsyncCalledWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcActor_CallAsyncCalledWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); final CompletableFuture contextClassLoader = testEndpoint.doCallAsync(); @@ -271,8 +270,8 @@ void testAkkaRpcActor_CallAsyncCalledWithFlinkContextClassLoader() throws Except } @Test - void testAkkaRpcActor_RunAsyncCalledWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcActor_RunAsyncCalledWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); final CompletableFuture contextClassLoader = testEndpoint.doRunAsync(); @@ -281,12 +280,12 @@ void testAkkaRpcActor_RunAsyncCalledWithFlinkContextClassLoader() throws Excepti } @Test - void testAkkaRpcActor_RPCReturningVoidCalledWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcActor_RPCReturningVoidCalledWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); final TestEndpointGateway testEndpointGateway = - akkaRpcService + pekkoRpcService .connect(testEndpoint.getAddress(), TestEndpointGateway.class) .get(); testEndpointGateway.doSomethingWithoutReturningAnything(); @@ -296,12 +295,12 @@ void testAkkaRpcActor_RPCReturningVoidCalledWithFlinkContextClassLoader() throws } @Test - void testAkkaRpcActor_RPCCalledWithFlinkContextClassLoader() throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcActor_RPCCalledWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); final TestEndpointGateway testEndpointGateway = - akkaRpcService + pekkoRpcService .connect(testEndpoint.getAddress(), TestEndpointGateway.class) .get(); final ClassLoader contextClassLoader = @@ -311,13 +310,12 @@ void testAkkaRpcActor_RPCCalledWithFlinkContextClassLoader() throws Exception { } @Test - void testAkkaRpcInvocationHandler_RPCFutureCompletedWithFlinkContextClassLoader() - throws Exception { - try (final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService)) { + void testRpcInvocationHandler_RPCFutureCompletedWithFlinkContextClassLoader() throws Exception { + try (final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService)) { testEndpoint.start(); final TestEndpointGateway testEndpointGateway = - akkaRpcService + pekkoRpcService .connect(testEndpoint.getAddress(), TestEndpointGateway.class) .get(); final CompletableFuture contextClassLoader = @@ -337,36 +335,36 @@ void testAkkaRpcInvocationHandler_RPCFutureCompletedWithFlinkContextClassLoader( } @Test - void testAkkaRpcInvocationHandler_ContextClassLoaderUsedForDeserialization() throws Exception { + void testRpcInvocationHandler_ContextClassLoaderUsedForDeserialization() throws Exception { // setup 2 actor systems and rpc services that support remote connections (for which RPCs go // through serialization) - final AkkaRpcService serverAkkaRpcService = - new AkkaRpcService( - AkkaUtils.createActorSystem( + final PekkoRpcService serverPekkoRpcService = + new PekkoRpcService( + PekkoUtils.createActorSystem( "serverActorSystem", - AkkaUtils.getAkkaConfig( + PekkoUtils.getConfig( new Configuration(), new HostAndPort("localhost", 0))), - AkkaRpcServiceConfiguration.defaultConfiguration()); + PekkoRpcServiceConfiguration.defaultConfiguration()); - final AkkaRpcService clientAkkaRpcService = - new AkkaRpcService( - AkkaUtils.createActorSystem( + final PekkoRpcService clientPekkoRpcService = + new PekkoRpcService( + PekkoUtils.createActorSystem( "clientActorSystem", - AkkaUtils.getAkkaConfig( + PekkoUtils.getConfig( new Configuration(), new HostAndPort("localhost", 0))), - AkkaRpcServiceConfiguration.defaultConfiguration(), + PekkoRpcServiceConfiguration.defaultConfiguration(), pretendFlinkClassLoader); try { final TestEndpoint rpcEndpoint = - new TestEndpoint(serverAkkaRpcService, new PickyObject()); + new TestEndpoint(serverPekkoRpcService, new PickyObject()); rpcEndpoint.start(); final TestEndpointGateway rpcGateway = rpcEndpoint.getSelfGateway(TestEndpointGateway.class); final TestEndpointGateway connect = - clientAkkaRpcService + clientPekkoRpcService .connect(rpcGateway.getAddress(), TestEndpointGateway.class) .get(); @@ -374,15 +372,15 @@ void testAkkaRpcInvocationHandler_ContextClassLoaderUsedForDeserialization() thr // exception connect.getPickyObject().get(); } finally { - RpcUtils.terminateRpcService(clientAkkaRpcService); - RpcUtils.terminateRpcService(serverAkkaRpcService); + RpcUtils.terminateRpcService(clientPekkoRpcService); + RpcUtils.terminateRpcService(serverPekkoRpcService); } } @Test void testSupervisorActor_TerminationFutureCompletedWithFlinkContextClassLoader() throws Exception { - final TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); + final TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService); testEndpoint.start(); final ClassLoader contextClassLoader = diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MainThreadValidationTest.java similarity index 83% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MainThreadValidationTest.java index f613d9358bb46..14a3e111d8bdf 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MainThreadValidationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -27,7 +27,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** - * Tests that the {@link AkkaRpcService} runs all RPCs in the {@link AkkaRpcActor}'s main thread. + * Tests that the {@link PekkoRpcService} runs all RPCs in the {@link PekkoRpcActor}'s main thread. */ class MainThreadValidationTest { @@ -42,13 +42,13 @@ void failIfNotInMainThread() throws Exception { } // actual test - AkkaRpcService akkaRpcService = - new AkkaRpcService( - AkkaUtils.createDefaultActorSystem(), - AkkaRpcServiceConfiguration.defaultConfiguration()); + PekkoRpcService pekkoRpcService = + new PekkoRpcService( + PekkoUtils.createDefaultActorSystem(), + PekkoRpcServiceConfiguration.defaultConfiguration()); try { - TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); + TestEndpoint testEndpoint = new TestEndpoint(pekkoRpcService); testEndpoint.start(); // this works, because it is executed as an RPC call @@ -60,7 +60,7 @@ void failIfNotInMainThread() throws Exception { testEndpoint.closeAsync(); } finally { - akkaRpcService.closeAsync().get(); + pekkoRpcService.closeAsync().get(); } } diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MessageSerializationTest.java similarity index 83% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MessageSerializationTest.java index cfeab0387fb08..5ddaae05708e4 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/MessageSerializationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -41,10 +41,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests that akka rpc invocation messages are properly serialized and errors reported. */ +/** Tests that pekko rpc invocation messages are properly serialized and errors reported. */ class MessageSerializationTest { - private static RpcService akkaRpcService1; - private static RpcService akkaRpcService2; + private static RpcService rpcService1; + private static RpcService rpcService2; private static final int maxFrameSize = 32000; @@ -53,11 +53,11 @@ static void setup() throws Exception { Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.FRAMESIZE, maxFrameSize + "b"); - akkaRpcService1 = - AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) + rpcService1 = + PekkoRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) .createAndStart(); - akkaRpcService2 = - AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) + rpcService2 = + PekkoRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) .createAndStart(); } @@ -65,8 +65,8 @@ static void setup() throws Exception { static void teardown() throws InterruptedException, ExecutionException, TimeoutException { final Collection> terminationFutures = new ArrayList<>(2); - terminationFutures.add(akkaRpcService1.closeAsync()); - terminationFutures.add(akkaRpcService2.closeAsync()); + terminationFutures.add(rpcService1.closeAsync()); + terminationFutures.add(rpcService2.closeAsync()); FutureUtils.waitForAll(terminationFutures).get(); } @@ -75,7 +75,7 @@ static void teardown() throws InterruptedException, ExecutionException, TimeoutE @Test void testNonSerializableLocalMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); - TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + TestEndpoint testEndpoint = new TestEndpoint(rpcService1, linkedBlockingQueue); testEndpoint.start(); TestGateway testGateway = testEndpoint.getSelfGateway(TestGateway.class); @@ -96,12 +96,12 @@ void testNonSerializableLocalMessageTransfer() throws Exception { void testNonSerializableRemoteMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); - TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + TestEndpoint testEndpoint = new TestEndpoint(rpcService1, linkedBlockingQueue); testEndpoint.start(); String address = testEndpoint.getAddress(); - TestGateway remoteGateway = akkaRpcService2.connect(address, TestGateway.class).get(); + TestGateway remoteGateway = rpcService2.connect(address, TestGateway.class).get(); assertThatThrownBy(() -> remoteGateway.foobar(new Object())) .isInstanceOf(IOException.class); @@ -112,13 +112,13 @@ void testNonSerializableRemoteMessageTransfer() throws Exception { void testSerializableRemoteMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); - TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + TestEndpoint testEndpoint = new TestEndpoint(rpcService1, linkedBlockingQueue); testEndpoint.start(); String address = testEndpoint.getAddress(); CompletableFuture remoteGatewayFuture = - akkaRpcService2.connect(address, TestGateway.class); + rpcService2.connect(address, TestGateway.class); TestGateway remoteGateway = remoteGatewayFuture.get(); @@ -134,12 +134,12 @@ void testSerializableRemoteMessageTransfer() throws Exception { void testMaximumFramesizeRemoteMessageTransfer() throws Throwable { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); - TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + TestEndpoint testEndpoint = new TestEndpoint(rpcService1, linkedBlockingQueue); testEndpoint.start(); String address = testEndpoint.getAddress(); - TestGateway remoteGateway = akkaRpcService2.connect(address, TestGateway.class).get(); + TestGateway remoteGateway = rpcService2.connect(address, TestGateway.class).get(); int bufferSize = maxFrameSize + 1; byte[] buffer = new byte[bufferSize]; diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaActorSystemTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoActorSystemTest.java similarity index 78% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaActorSystemTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoActorSystemTest.java index cd08baad57122..59aac7ff0709e 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaActorSystemTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoActorSystemTest.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Props; -import akka.actor.Terminated; -import akka.japi.pf.ReceiveBuilder; -import akka.pattern.Patterns; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Props; +import org.apache.pekko.actor.Terminated; +import org.apache.pekko.japi.pf.ReceiveBuilder; +import org.apache.pekko.pattern.Patterns; import org.junit.jupiter.api.Test; import java.time.Duration; @@ -37,15 +37,12 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** - * Tests for the {@link akka.actor.ActorSystem} instantiated through {@link - * org.apache.flink.runtime.rpc.akka.AkkaUtils}. - */ -class AkkaActorSystemTest { +/** Tests for the {@link ActorSystem} instantiated through {@link PekkoUtils}. */ +class PekkoActorSystemTest { @Test void shutsDownOnActorFailure() { - final ActorSystem actorSystem = AkkaUtils.createLocalActorSystem(new Configuration()); + final ActorSystem actorSystem = PekkoUtils.createLocalActorSystem(new Configuration()); try { final CompletableFuture terminationFuture = @@ -59,13 +56,13 @@ void shutsDownOnActorFailure() { // make sure that the ActorSystem shuts down terminationFuture.join(); } finally { - AkkaUtils.terminateActorSystem(actorSystem).join(); + PekkoUtils.terminateActorSystem(actorSystem).join(); } } @Test void askTerminatedActorFailsWithRecipientTerminatedException() { - final ActorSystem actorSystem = AkkaUtils.createLocalActorSystem(new Configuration()); + final ActorSystem actorSystem = PekkoUtils.createLocalActorSystem(new Configuration()); final Duration timeout = Duration.ofSeconds(10L); try { @@ -78,9 +75,9 @@ void askTerminatedActorFailsWithRecipientTerminatedException() { assertThatThrownBy(() -> result.toCompletableFuture().get()) .extracting(ExceptionUtils::stripExecutionException) - .matches(AkkaRpcServiceUtils::isRecipientTerminatedException); + .matches(PekkoRpcServiceUtils::isRecipientTerminatedException); } finally { - AkkaUtils.terminateActorSystem(actorSystem).join(); + PekkoUtils.terminateActorSystem(actorSystem).join(); } } diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorHandshakeTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorHandshakeTest.java similarity index 63% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorHandshakeTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorHandshakeTest.java index dfc5ba4d9f300..b2e659dacdbd2 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorHandshakeTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorHandshakeTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcUtils; @@ -24,7 +24,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.concurrent.FutureUtils; -import akka.actor.ActorSystem; +import org.apache.pekko.actor.ActorSystem; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -37,54 +37,54 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the handshake between rpc endpoints. */ -class AkkaRpcActorHandshakeTest { +class PekkoRpcActorHandshakeTest { - private static AkkaRpcService akkaRpcService1; - private static AkkaRpcService akkaRpcService2; - private static WrongVersionAkkaRpcService wrongVersionAkkaRpcService; + private static PekkoRpcService rpcService1; + private static PekkoRpcService rpcService2; + private static WrongVersionPekkoRpcService wrongVersionRpcService; @BeforeAll static void setupClass() { - final ActorSystem actorSystem1 = AkkaUtils.createDefaultActorSystem(); - final ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem(); - final ActorSystem wrongVersionActorSystem = AkkaUtils.createDefaultActorSystem(); - - AkkaRpcServiceConfiguration akkaRpcServiceConfig = - AkkaRpcServiceConfiguration.defaultConfiguration(); - akkaRpcService1 = new AkkaRpcService(actorSystem1, akkaRpcServiceConfig); - akkaRpcService2 = new AkkaRpcService(actorSystem2, akkaRpcServiceConfig); - wrongVersionAkkaRpcService = - new WrongVersionAkkaRpcService( + final ActorSystem actorSystem1 = PekkoUtils.createDefaultActorSystem(); + final ActorSystem actorSystem2 = PekkoUtils.createDefaultActorSystem(); + final ActorSystem wrongVersionActorSystem = PekkoUtils.createDefaultActorSystem(); + + PekkoRpcServiceConfiguration rpcServiceConfig = + PekkoRpcServiceConfiguration.defaultConfiguration(); + rpcService1 = new PekkoRpcService(actorSystem1, rpcServiceConfig); + rpcService2 = new PekkoRpcService(actorSystem2, rpcServiceConfig); + wrongVersionRpcService = + new WrongVersionPekkoRpcService( wrongVersionActorSystem, - AkkaRpcServiceConfiguration.defaultConfiguration()); + PekkoRpcServiceConfiguration.defaultConfiguration()); } @AfterAll static void teardownClass() throws Exception { final Collection> terminationFutures = new ArrayList<>(3); - terminationFutures.add(akkaRpcService1.closeAsync()); - terminationFutures.add(akkaRpcService2.closeAsync()); - terminationFutures.add(wrongVersionAkkaRpcService.closeAsync()); + terminationFutures.add(rpcService1.closeAsync()); + terminationFutures.add(rpcService2.closeAsync()); + terminationFutures.add(wrongVersionRpcService.closeAsync()); FutureUtils.waitForAll(terminationFutures).get(); } @Test void testVersionMatchBetweenRpcComponents() throws Exception { - AkkaRpcActorTest.DummyRpcEndpoint rpcEndpoint = - new AkkaRpcActorTest.DummyRpcEndpoint(akkaRpcService1); + PekkoRpcActorTest.DummyRpcEndpoint rpcEndpoint = + new PekkoRpcActorTest.DummyRpcEndpoint(rpcService1); final int value = 42; rpcEndpoint.setFoobar(value); rpcEndpoint.start(); try { - final AkkaRpcActorTest.DummyRpcGateway dummyRpcGateway = - akkaRpcService2 + final PekkoRpcActorTest.DummyRpcGateway dummyRpcGateway = + rpcService2 .connect( rpcEndpoint.getAddress(), - AkkaRpcActorTest.DummyRpcGateway.class) + PekkoRpcActorTest.DummyRpcGateway.class) .get(); assertThat(dummyRpcGateway.foobar().get()).isEqualTo(value); @@ -95,18 +95,18 @@ void testVersionMatchBetweenRpcComponents() throws Exception { @Test void testVersionMismatchBetweenRpcComponents() throws Exception { - AkkaRpcActorTest.DummyRpcEndpoint rpcEndpoint = - new AkkaRpcActorTest.DummyRpcEndpoint(akkaRpcService1); + PekkoRpcActorTest.DummyRpcEndpoint rpcEndpoint = + new PekkoRpcActorTest.DummyRpcEndpoint(rpcService1); rpcEndpoint.start(); try { assertThatThrownBy( () -> - wrongVersionAkkaRpcService + wrongVersionRpcService .connect( rpcEndpoint.getAddress(), - AkkaRpcActorTest.DummyRpcGateway.class) + PekkoRpcActorTest.DummyRpcGateway.class) .get()) .extracting(ExceptionUtils::stripExecutionException) .isInstanceOf(HandshakeException.class); @@ -121,13 +121,13 @@ void testVersionMismatchBetweenRpcComponents() throws Exception { */ @Test void testWrongGatewayEndpointConnection() throws Exception { - AkkaRpcActorTest.DummyRpcEndpoint rpcEndpoint = - new AkkaRpcActorTest.DummyRpcEndpoint(akkaRpcService1); + PekkoRpcActorTest.DummyRpcEndpoint rpcEndpoint = + new PekkoRpcActorTest.DummyRpcEndpoint(rpcService1); rpcEndpoint.start(); CompletableFuture futureGateway = - akkaRpcService2.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); + rpcService2.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); try { assertThatThrownBy(() -> futureGateway.get()) @@ -139,10 +139,10 @@ void testWrongGatewayEndpointConnection() throws Exception { } } - private static class WrongVersionAkkaRpcService extends AkkaRpcService { + private static class WrongVersionPekkoRpcService extends PekkoRpcService { - WrongVersionAkkaRpcService( - ActorSystem actorSystem, AkkaRpcServiceConfiguration configuration) { + WrongVersionPekkoRpcService( + ActorSystem actorSystem, PekkoRpcServiceConfiguration configuration) { super(actorSystem, configuration); } diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorOversizedResponseMessageTest.java similarity index 92% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorOversizedResponseMessageTest.java index f66f6ca688d96..1a7f2dfdf4cbd 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorOversizedResponseMessageTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -25,7 +25,6 @@ import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; import org.apache.flink.runtime.rpc.exceptions.RpcException; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.function.FunctionWithException; @@ -41,8 +40,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the over sized response message handling of the {@link AkkaRpcActor}. */ -class AkkaRpcActorOversizedResponseMessageTest { +/** Tests for the over sized response message handling of the {@link PekkoRpcActor}. */ +class PekkoRpcActorOversizedResponseMessageTest { private static final int FRAMESIZE = 32000; @@ -62,10 +61,10 @@ static void setupClass() throws Exception { configuration.setString(AkkaOptions.FRAMESIZE, FRAMESIZE + " b"); rpcService1 = - AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) + PekkoRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) .createAndStart(); rpcService2 = - AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) + PekkoRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0) .createAndStart(); } @@ -80,9 +79,9 @@ void testOverSizedResponseMsgAsync() throws Exception { () -> runRemoteMessageResponseTest( OVERSIZED_PAYLOAD, this::requestMessageAsync)) - .hasCauseInstanceOf(AkkaRpcException.class) + .hasCauseInstanceOf(RpcException.class) .extracting(ExceptionUtils::stripExecutionException) - .isInstanceOf(AkkaRpcException.class) + .isInstanceOf(RpcException.class) .extracting(Throwable::getMessage) .satisfies(message -> assertThat(message).contains(String.valueOf(FRAMESIZE))); } @@ -108,7 +107,7 @@ void testOverSizedResponseMsgSync() throws Exception { OVERSIZED_PAYLOAD, MessageRpcGateway::messageSync)) .satisfies( FlinkAssertions.anyCauseMatches( - AkkaRpcException.class, String.valueOf(FRAMESIZE))); + RpcException.class, String.valueOf(FRAMESIZE))); } /** diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorTest.java similarity index 88% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorTest.java index c95fee2630510..3015d4593de17 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcActorTest.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.Local; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -41,8 +41,8 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.util.concurrent.FutureUtils; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSystem; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -67,10 +67,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the {@link AkkaRpcActor}. */ -class AkkaRpcActorTest { +/** Tests for the {@link PekkoRpcActor}. */ +class PekkoRpcActorTest { - private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActorTest.class); + private static final Logger LOG = LoggerFactory.getLogger(PekkoRpcActorTest.class); // ------------------------------------------------------------------------ // shared test members @@ -78,19 +78,19 @@ class AkkaRpcActorTest { private static Duration timeout = Duration.ofSeconds(10L); - private static AkkaRpcService akkaRpcService; + private static PekkoRpcService pekkoRpcService; @BeforeAll static void setup() { - akkaRpcService = - new AkkaRpcService( - AkkaUtils.createLocalActorSystem(new Configuration()), - AkkaRpcServiceConfiguration.defaultConfiguration()); + pekkoRpcService = + new PekkoRpcService( + PekkoUtils.createLocalActorSystem(new Configuration()), + PekkoRpcServiceConfiguration.defaultConfiguration()); } @AfterAll static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { - RpcUtils.terminateRpcService(akkaRpcService); + RpcUtils.terminateRpcService(pekkoRpcService); } /** @@ -100,10 +100,10 @@ static void shutdown() throws InterruptedException, ExecutionException, TimeoutE */ @Test void testAddressResolution() throws Exception { - DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(pekkoRpcService); CompletableFuture futureRpcGateway = - akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); + pekkoRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); DummyRpcGateway rpcGateway = futureRpcGateway.get(); @@ -117,21 +117,21 @@ void testAddressResolution() throws Exception { @Test void testFailingAddressResolution() throws Exception { CompletableFuture futureRpcGateway = - akkaRpcService.connect("foobar", DummyRpcGateway.class); + pekkoRpcService.connect("foobar", DummyRpcGateway.class); assertThatThrownBy(() -> futureRpcGateway.get()) .hasCauseInstanceOf(RpcConnectionException.class); } /** - * Tests that the {@link AkkaRpcActor} discards messages until the corresponding {@link + * Tests that the {@link PekkoRpcActor} discards messages until the corresponding {@link * RpcEndpoint} has been started. */ @Test void testMessageDiscarding() throws Exception { int expectedValue = 1337; - DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(pekkoRpcService); DummyRpcGateway rpcGateway = rpcEndpoint.getSelfGateway(DummyRpcGateway.class); @@ -166,14 +166,14 @@ void testMessageDiscarding() throws Exception { */ @Test void testRpcEndpointTerminationFuture() throws Exception { - final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(pekkoRpcService); rpcEndpoint.start(); CompletableFuture terminationFuture = rpcEndpoint.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); - CompletableFuture.runAsync(rpcEndpoint::closeAsync, akkaRpcService.getScheduledExecutor()); + CompletableFuture.runAsync(rpcEndpoint::closeAsync, pekkoRpcService.getScheduledExecutor()); // wait until the rpc endpoint has terminated terminationFuture.get(); @@ -181,7 +181,7 @@ void testRpcEndpointTerminationFuture() throws Exception { @Test void testExceptionPropagation() throws Exception { - ExceptionalEndpoint rpcEndpoint = new ExceptionalEndpoint(akkaRpcService); + ExceptionalEndpoint rpcEndpoint = new ExceptionalEndpoint(pekkoRpcService); rpcEndpoint.start(); ExceptionalGateway rpcGateway = rpcEndpoint.getSelfGateway(ExceptionalGateway.class); @@ -198,7 +198,7 @@ void testExceptionPropagation() throws Exception { @Test void testExceptionPropagationFuturePiping() throws Exception { - ExceptionalFutureEndpoint rpcEndpoint = new ExceptionalFutureEndpoint(akkaRpcService); + ExceptionalFutureEndpoint rpcEndpoint = new ExceptionalFutureEndpoint(pekkoRpcService); rpcEndpoint.start(); ExceptionalGateway rpcGateway = rpcEndpoint.getSelfGateway(ExceptionalGateway.class); @@ -211,39 +211,39 @@ void testExceptionPropagationFuturePiping() throws Exception { } /** - * Tests that the AkkaInvocationHandler properly fails the returned future if the response - * cannot be deserialized. + * Tests that the {@link PekkoInvocationHandler} properly fails the returned future if the + * response cannot be deserialized. */ @Test void testResultFutureFailsOnDeserializationError() throws Exception { // setup 2 actor systems and rpc services that support remote connections (for which RPCs go // through serialization) - final AkkaRpcService serverAkkaRpcService = - new AkkaRpcService( - AkkaUtils.createActorSystem( + final PekkoRpcService serverPekkoRpcService = + new PekkoRpcService( + PekkoUtils.createActorSystem( "serverActorSystem", - AkkaUtils.getAkkaConfig( + PekkoUtils.getConfig( new Configuration(), new HostAndPort("localhost", 0))), - AkkaRpcServiceConfiguration.defaultConfiguration()); + PekkoRpcServiceConfiguration.defaultConfiguration()); - final AkkaRpcService clientAkkaRpcService = - new AkkaRpcService( - AkkaUtils.createActorSystem( + final PekkoRpcService clientPekkoRpcService = + new PekkoRpcService( + PekkoUtils.createActorSystem( "clientActorSystem", - AkkaUtils.getAkkaConfig( + PekkoUtils.getConfig( new Configuration(), new HostAndPort("localhost", 0))), - AkkaRpcServiceConfiguration.defaultConfiguration()); + PekkoRpcServiceConfiguration.defaultConfiguration()); try { final DeserializatonFailingEndpoint rpcEndpoint = - new DeserializatonFailingEndpoint(serverAkkaRpcService); + new DeserializatonFailingEndpoint(serverPekkoRpcService); rpcEndpoint.start(); final DeserializatonFailingGateway rpcGateway = rpcEndpoint.getSelfGateway(DeserializatonFailingGateway.class); final DeserializatonFailingGateway connect = - clientAkkaRpcService + clientPekkoRpcService .connect(rpcGateway.getAddress(), DeserializatonFailingGateway.class) .get(); @@ -251,8 +251,8 @@ void testResultFutureFailsOnDeserializationError() throws Exception { .eventuallyFailsWith(ExecutionException.class) .withCauseInstanceOf(RpcException.class); } finally { - RpcUtils.terminateRpcService(clientAkkaRpcService); - RpcUtils.terminateRpcService(serverAkkaRpcService); + RpcUtils.terminateRpcService(clientPekkoRpcService); + RpcUtils.terminateRpcService(serverPekkoRpcService); } } @@ -260,7 +260,7 @@ void testResultFutureFailsOnDeserializationError() throws Exception { @Test void testOnStopExceptionPropagation() throws Exception { FailingOnStopEndpoint rpcEndpoint = - new FailingOnStopEndpoint(akkaRpcService, "FailingOnStopEndpoint"); + new FailingOnStopEndpoint(pekkoRpcService, "FailingOnStopEndpoint"); rpcEndpoint.start(); CompletableFuture terminationFuture = rpcEndpoint.closeAsync(); @@ -273,7 +273,7 @@ void testOnStopExceptionPropagation() throws Exception { @Test void testOnStopExecutedByMainThread() throws Exception { SimpleRpcEndpoint simpleRpcEndpoint = - new SimpleRpcEndpoint(akkaRpcService, "SimpleRpcEndpoint"); + new SimpleRpcEndpoint(pekkoRpcService, "SimpleRpcEndpoint"); simpleRpcEndpoint.start(); CompletableFuture terminationFuture = simpleRpcEndpoint.closeAsync(); @@ -283,13 +283,13 @@ void testOnStopExecutedByMainThread() throws Exception { terminationFuture.get(); } - /** Tests that actors are properly terminated when the AkkaRpcService is shut down. */ + /** Tests that actors are properly terminated when the {@link PekkoRpcService} is shut down. */ @Test void testActorTerminationWhenServiceShutdown() throws Exception { - final ActorSystem rpcActorSystem = AkkaUtils.createDefaultActorSystem(); + final ActorSystem rpcActorSystem = PekkoUtils.createDefaultActorSystem(); final RpcService rpcService = - new AkkaRpcService( - rpcActorSystem, AkkaRpcServiceConfiguration.defaultConfiguration()); + new PekkoRpcService( + rpcActorSystem, PekkoRpcServiceConfiguration.defaultConfiguration()); try { SimpleRpcEndpoint rpcEndpoint = @@ -304,19 +304,19 @@ void testActorTerminationWhenServiceShutdown() throws Exception { terminationFuture.get(); } finally { rpcActorSystem.terminate(); - AkkaFutureUtils.toJava(rpcActorSystem.whenTerminated()).get(); + ScalaFutureUtils.toJava(rpcActorSystem.whenTerminated()).get(); } } /** - * Tests that the {@link AkkaRpcActor} only completes after the asynchronous post stop action + * Tests that the {@link PekkoRpcActor} only completes after the asynchronous post stop action * has completed. */ @Test void testActorTerminationWithAsynchronousOnStopAction() throws Exception { final CompletableFuture onStopFuture = new CompletableFuture<>(); final AsynchronousOnStopEndpoint endpoint = - new AsynchronousOnStopEndpoint(akkaRpcService, onStopFuture); + new AsynchronousOnStopEndpoint(pekkoRpcService, onStopFuture); try { endpoint.start(); @@ -341,7 +341,7 @@ void testActorTerminationWithAsynchronousOnStopAction() throws Exception { @Test void testMainThreadExecutionOnStop() throws Exception { final MainThreadExecutorOnStopEndpoint endpoint = - new MainThreadExecutorOnStopEndpoint(akkaRpcService); + new MainThreadExecutorOnStopEndpoint(pekkoRpcService); try { endpoint.start(); @@ -356,10 +356,10 @@ void testMainThreadExecutionOnStop() throws Exception { /** Tests that when the onStop future completes that no other messages will be processed. */ @Test - void testOnStopFutureCompletionDirectlyTerminatesAkkaRpcActor() throws Exception { + void testOnStopFutureCompletionDirectlyTerminatesRpcActor() throws Exception { final CompletableFuture onStopFuture = new CompletableFuture<>(); final TerminatingAfterOnStopFutureCompletionEndpoint endpoint = - new TerminatingAfterOnStopFutureCompletionEndpoint(akkaRpcService, onStopFuture); + new TerminatingAfterOnStopFutureCompletionEndpoint(pekkoRpcService, onStopFuture); try { endpoint.start(); @@ -407,7 +407,7 @@ void testOnStopFutureCompletionDirectlyTerminatesAkkaRpcActor() throws Exception */ @Test void testOnStartIsCalledWhenRpcEndpointStarts() throws Exception { - final OnStartEndpoint onStartEndpoint = new OnStartEndpoint(akkaRpcService, null); + final OnStartEndpoint onStartEndpoint = new OnStartEndpoint(pekkoRpcService, null); try { onStartEndpoint.start(); @@ -421,7 +421,7 @@ void testOnStartIsCalledWhenRpcEndpointStarts() throws Exception { @Test void testOnStartFails() throws Exception { final FlinkException testException = new FlinkException("Test exception"); - final OnStartEndpoint onStartEndpoint = new OnStartEndpoint(akkaRpcService, testException); + final OnStartEndpoint onStartEndpoint = new OnStartEndpoint(pekkoRpcService, testException); onStartEndpoint.start(); onStartEndpoint.awaitUntilOnStartCalled(); @@ -441,12 +441,13 @@ void testOnStartFails() throws Exception { void callsOnStopOnlyOnce() throws Exception { final CompletableFuture onStopFuture = new CompletableFuture<>(); final OnStopCountingRpcEndpoint endpoint = - new OnStopCountingRpcEndpoint(akkaRpcService, onStopFuture); + new OnStopCountingRpcEndpoint(pekkoRpcService, onStopFuture); try { endpoint.start(); - final AkkaBasedEndpoint selfGateway = endpoint.getSelfGateway(AkkaBasedEndpoint.class); + final PekkoBasedEndpoint selfGateway = + endpoint.getSelfGateway(PekkoBasedEndpoint.class); // try to terminate the actor twice selfGateway.getActorRef().tell(ControlMessages.TERMINATE, ActorRef.noSender()); @@ -469,14 +470,14 @@ void callsOnStopOnlyOnce() throws Exception { void canReuseEndpointNameAfterTermination() throws Exception { final String endpointName = "not_unique"; try (SimpleRpcEndpoint simpleRpcEndpoint1 = - new SimpleRpcEndpoint(akkaRpcService, endpointName)) { + new SimpleRpcEndpoint(pekkoRpcService, endpointName)) { simpleRpcEndpoint1.start(); simpleRpcEndpoint1.closeAsync().join(); try (SimpleRpcEndpoint simpleRpcEndpoint2 = - new SimpleRpcEndpoint(akkaRpcService, endpointName)) { + new SimpleRpcEndpoint(pekkoRpcService, endpointName)) { simpleRpcEndpoint2.start(); assertThat(simpleRpcEndpoint2.getAddress()) @@ -488,7 +489,7 @@ void canReuseEndpointNameAfterTermination() throws Exception { @Test void terminationFutureDoesNotBlockRpcEndpointCreation() throws Exception { try (final SimpleRpcEndpoint simpleRpcEndpoint = - new SimpleRpcEndpoint(akkaRpcService, "foobar")) { + new SimpleRpcEndpoint(pekkoRpcService, "foobar")) { final CompletableFuture terminationFuture = simpleRpcEndpoint.getTerminationFuture(); @@ -496,7 +497,7 @@ void terminationFutureDoesNotBlockRpcEndpointCreation() throws Exception { // completing the termination future won't block the RpcService final CompletableFuture foobar2 = terminationFuture.thenApply( - ignored -> new SimpleRpcEndpoint(akkaRpcService, "foobar2")); + ignored -> new SimpleRpcEndpoint(pekkoRpcService, "foobar2")); simpleRpcEndpoint.closeAsync(); @@ -506,7 +507,7 @@ void terminationFutureDoesNotBlockRpcEndpointCreation() throws Exception { } @Test - void resolvesRunningAkkaRpcActor() throws Exception { + void resolvesRunningRpcActor() throws Exception { final String endpointName = "foobar"; try (RpcEndpoint simpleRpcEndpoint1 = createRpcEndpointWithRandomNameSuffix(endpointName); @@ -516,22 +517,22 @@ void resolvesRunningAkkaRpcActor() throws Exception { simpleRpcEndpoint1.closeAsync().join(); final String wildcardName = RpcServiceUtils.createWildcardName(endpointName); - final String wildcardAddress = AkkaRpcServiceUtils.getLocalRpcUrl(wildcardName); + final String wildcardAddress = PekkoRpcServiceUtils.getLocalRpcUrl(wildcardName); final RpcGateway rpcGateway = - akkaRpcService.connect(wildcardAddress, RpcGateway.class).join(); + pekkoRpcService.connect(wildcardAddress, RpcGateway.class).join(); assertThat(rpcGateway.getAddress()).isEqualTo(simpleRpcEndpoint2.getAddress()); } } private RpcEndpoint createRpcEndpointWithRandomNameSuffix(String prefix) { - return new SimpleRpcEndpoint(akkaRpcService, RpcServiceUtils.createRandomName(prefix)); + return new SimpleRpcEndpoint(pekkoRpcService, RpcServiceUtils.createRandomName(prefix)); } @Test void canRespondWithNullValueLocally() throws Exception { try (final NullRespondingEndpoint nullRespondingEndpoint = - new NullRespondingEndpoint(akkaRpcService)) { + new NullRespondingEndpoint(pekkoRpcService)) { nullRespondingEndpoint.start(); final NullRespondingGateway selfGateway = @@ -546,7 +547,7 @@ void canRespondWithNullValueLocally() throws Exception { @Test void canRespondWithSynchronousNullValueLocally() throws Exception { try (final NullRespondingEndpoint nullRespondingEndpoint = - new NullRespondingEndpoint(akkaRpcService)) { + new NullRespondingEndpoint(pekkoRpcService)) { nullRespondingEndpoint.start(); final NullRespondingGateway selfGateway = @@ -561,7 +562,7 @@ void canRespondWithSynchronousNullValueLocally() throws Exception { @Test void canRespondWithSerializedValueLocally() throws Exception { try (final SerializedValueRespondingEndpoint endpoint = - new SerializedValueRespondingEndpoint(akkaRpcService)) { + new SerializedValueRespondingEndpoint(pekkoRpcService)) { endpoint.start(); final SerializedValueRespondingGateway selfGateway = @@ -587,15 +588,15 @@ void canRespondWithSerializedValueLocally() throws Exception { * upper bound exists in the first place. * *

There are various failure points for this test, including the scheduling from the {@link - * RpcEndpoint} to the {@link AkkaInvocationHandler}, the conversion of these calls by the + * RpcEndpoint} to the {@link PekkoInvocationHandler}, the conversion of these calls by the * handler into Call-/RunAsync messages, the handling of said messages by the {@link - * AkkaRpcActor} and in the case of RunAsync the actual scheduling by the underlying actor + * PekkoRpcActor} and in the case of RunAsync the actual scheduling by the underlying actor * system. This isn't an ideal test setup, but these components are difficult to test in * isolation. */ @Test void testScheduling() throws ExecutionException, InterruptedException { - final SchedulingRpcEndpoint endpoint = new SchedulingRpcEndpoint(akkaRpcService); + final SchedulingRpcEndpoint endpoint = new SchedulingRpcEndpoint(pekkoRpcService); endpoint.start(); diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceTest.java similarity index 77% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceTest.java index 0b3f51bfec0eb..2fb7bc8d1c773 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcServiceTest.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.exceptions.RecipientUnreachableException; @@ -28,8 +28,8 @@ import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ScheduledExecutor; -import akka.actor.ActorSystem; -import akka.actor.Terminated; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Terminated; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -52,8 +52,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the {@link AkkaRpcService}. */ -class AkkaRpcServiceTest { +/** Tests for the {@link PekkoRpcService}. */ +class PekkoRpcServiceTest { // ------------------------------------------------------------------------ // shared test members @@ -61,26 +61,27 @@ class AkkaRpcServiceTest { private static ActorSystem actorSystem; - private static AkkaRpcService akkaRpcService; + private static PekkoRpcService pekkoRpcService; @BeforeAll static void setup() { - actorSystem = AkkaUtils.createDefaultActorSystem(); - akkaRpcService = - new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.defaultConfiguration()); + actorSystem = PekkoUtils.createDefaultActorSystem(); + pekkoRpcService = + new PekkoRpcService( + actorSystem, PekkoRpcServiceConfiguration.defaultConfiguration()); } @AfterAll static void shutdown() throws InterruptedException, ExecutionException, TimeoutException { - final CompletableFuture rpcTerminationFuture = akkaRpcService.closeAsync(); + final CompletableFuture rpcTerminationFuture = pekkoRpcService.closeAsync(); final CompletableFuture actorSystemTerminationFuture = - AkkaFutureUtils.toJava(actorSystem.terminate()); + ScalaFutureUtils.toJava(actorSystem.terminate()); FutureUtils.waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) .get(); actorSystem = null; - akkaRpcService = null; + pekkoRpcService = null; } // ------------------------------------------------------------------------ @@ -93,7 +94,7 @@ void testScheduleRunnable() throws Exception { final long start = System.nanoTime(); ScheduledFuture scheduledFuture = - akkaRpcService + pekkoRpcService .getScheduledExecutor() .schedule(latch::trigger, delay, TimeUnit.MILLISECONDS); @@ -107,26 +108,26 @@ void testScheduleRunnable() throws Exception { .isGreaterThanOrEqualTo(delay); } - /** Tests that the {@link AkkaRpcService} can execute runnables. */ + /** Tests that the {@link PekkoRpcService} can execute runnables. */ @Test void testExecuteRunnable() throws Exception { final OneShotLatch latch = new OneShotLatch(); - akkaRpcService.getScheduledExecutor().execute(latch::trigger); + pekkoRpcService.getScheduledExecutor().execute(latch::trigger); latch.await(30L, TimeUnit.SECONDS); } @Test void testGetAddress() { - assertThat(akkaRpcService.getAddress()) - .isEqualTo(AkkaUtils.getAddress(actorSystem).host().get()); + assertThat(pekkoRpcService.getAddress()) + .isEqualTo(PekkoUtils.getAddress(actorSystem).host().get()); } @Test void testGetPort() { - assertThat(akkaRpcService.getPort()) - .isEqualTo(AkkaUtils.getAddress(actorSystem).port().get()); + assertThat(pekkoRpcService.getPort()) + .isEqualTo(PekkoUtils.getAddress(actorSystem).port().get()); } /** @@ -135,7 +136,7 @@ void testGetPort() { */ @Test void testScheduledExecutorServiceSimpleSchedule() throws Exception { - ScheduledExecutor scheduledExecutor = akkaRpcService.getScheduledExecutor(); + ScheduledExecutor scheduledExecutor = pekkoRpcService.getScheduledExecutor(); final OneShotLatch latch = new OneShotLatch(); @@ -154,7 +155,7 @@ void testScheduledExecutorServiceSimpleSchedule() throws Exception { */ @Test void testScheduledExecutorServicePeriodicSchedule() throws Exception { - ScheduledExecutor scheduledExecutor = akkaRpcService.getScheduledExecutor(); + ScheduledExecutor scheduledExecutor = pekkoRpcService.getScheduledExecutor(); final int tries = 4; final long delay = 10L; @@ -187,7 +188,7 @@ void testScheduledExecutorServicePeriodicSchedule() throws Exception { */ @Test void testScheduledExecutorServiceWithFixedDelaySchedule() throws Exception { - ScheduledExecutor scheduledExecutor = akkaRpcService.getScheduledExecutor(); + ScheduledExecutor scheduledExecutor = pekkoRpcService.getScheduledExecutor(); final int tries = 4; final long delay = 10L; @@ -219,7 +220,7 @@ void testScheduledExecutorServiceWithFixedDelaySchedule() throws Exception { */ @Test void testScheduledExecutorServiceCancelWithFixedDelay() throws InterruptedException { - ScheduledExecutor scheduledExecutor = akkaRpcService.getScheduledExecutor(); + ScheduledExecutor scheduledExecutor = pekkoRpcService.getScheduledExecutor(); long delay = 10L; @@ -258,16 +259,18 @@ void testScheduledExecutorServiceCancelWithFixedDelay() throws InterruptedExcept .isInstanceOf(TimeoutException.class); } - /** Tests that the {@link AkkaRpcService} terminates all its RpcEndpoints when shutting down. */ + /** + * Tests that the {@link PekkoRpcService} terminates all its RpcEndpoints when shutting down. + */ @Test - void testAkkaRpcServiceShutDownWithRpcEndpoints() throws Exception { - final AkkaRpcService akkaRpcService = startAkkaRpcService(); + void testRpcServiceShutDownWithRpcEndpoints() throws Exception { + final PekkoRpcService pekkoRpcService = startRpcService(); try { final int numberActors = 5; final RpcServiceShutdownTestHelper rpcServiceShutdownTestHelper = - startStopNCountingAsynchronousOnStopEndpoints(akkaRpcService, numberActors); + startStopNCountingAsynchronousOnStopEndpoints(pekkoRpcService, numberActors); for (CompletableFuture onStopFuture : rpcServiceShutdownTestHelper.getStopFutures()) { @@ -275,24 +278,24 @@ void testAkkaRpcServiceShutDownWithRpcEndpoints() throws Exception { } rpcServiceShutdownTestHelper.waitForRpcServiceTermination(); - assertThat(akkaRpcService.getActorSystem().whenTerminated().isCompleted()).isTrue(); + assertThat(pekkoRpcService.getActorSystem().whenTerminated().isCompleted()).isTrue(); } finally { - RpcUtils.terminateRpcService(akkaRpcService); + RpcUtils.terminateRpcService(pekkoRpcService); } } /** - * Tests that {@link AkkaRpcService} terminates all its RpcEndpoints and also stops the + * Tests that {@link PekkoRpcService} terminates all its RpcEndpoints and also stops the * underlying {@link ActorSystem} if one of the RpcEndpoints fails while stopping. */ @Test - void testAkkaRpcServiceShutDownWithFailingRpcEndpoints() throws Exception { - final AkkaRpcService akkaRpcService = startAkkaRpcService(); + void testRpcServiceShutDownWithFailingRpcEndpoints() throws Exception { + final PekkoRpcService pekkoRpcService = startRpcService(); final int numberActors = 5; final RpcServiceShutdownTestHelper rpcServiceShutdownTestHelper = - startStopNCountingAsynchronousOnStopEndpoints(akkaRpcService, numberActors); + startStopNCountingAsynchronousOnStopEndpoints(pekkoRpcService, numberActors); final Iterator> iterator = rpcServiceShutdownTestHelper.getStopFutures().iterator(); @@ -306,20 +309,20 @@ void testAkkaRpcServiceShutDownWithFailingRpcEndpoints() throws Exception { assertThatThrownBy(rpcServiceShutdownTestHelper::waitForRpcServiceTermination) .satisfies(FlinkAssertions.anyCauseMatches(OnStopException.class)); - assertThat(akkaRpcService.getActorSystem().whenTerminated().isCompleted()).isTrue(); + assertThat(pekkoRpcService.getActorSystem().whenTerminated().isCompleted()).isTrue(); } @Test void failsRpcResultImmediatelyIfEndpointIsStopped() throws Exception { - try (final AkkaRpcActorTest.SerializedValueRespondingEndpoint endpoint = - new AkkaRpcActorTest.SerializedValueRespondingEndpoint(akkaRpcService)) { + try (final PekkoRpcActorTest.SerializedValueRespondingEndpoint endpoint = + new PekkoRpcActorTest.SerializedValueRespondingEndpoint(pekkoRpcService)) { endpoint.start(); - final AkkaRpcActorTest.SerializedValueRespondingGateway gateway = - akkaRpcService + final PekkoRpcActorTest.SerializedValueRespondingGateway gateway = + pekkoRpcService .connect( endpoint.getAddress(), - AkkaRpcActorTest.SerializedValueRespondingGateway.class) + PekkoRpcActorTest.SerializedValueRespondingGateway.class) .join(); endpoint.close(); @@ -331,7 +334,7 @@ void failsRpcResultImmediatelyIfEndpointIsStopped() throws Exception { } private static RpcServiceShutdownTestHelper startStopNCountingAsynchronousOnStopEndpoints( - AkkaRpcService akkaRpcService, int numberActors) throws InterruptedException { + PekkoRpcService pekkoRpcService, int numberActors) throws InterruptedException { final Collection> onStopFutures = new ArrayList<>(numberActors); final CountDownLatch countDownLatch = new CountDownLatch(numberActors); @@ -340,17 +343,17 @@ private static RpcServiceShutdownTestHelper startStopNCountingAsynchronousOnStop CompletableFuture onStopFuture = new CompletableFuture<>(); final CountingAsynchronousOnStopEndpoint endpoint = new CountingAsynchronousOnStopEndpoint( - akkaRpcService, onStopFuture, countDownLatch); + pekkoRpcService, onStopFuture, countDownLatch); endpoint.start(); onStopFutures.add(onStopFuture); } - CompletableFuture terminationFuture = akkaRpcService.closeAsync(); + CompletableFuture terminationFuture = pekkoRpcService.closeAsync(); countDownLatch.await(); assertThat(terminationFuture).isNotDone(); - assertThat(akkaRpcService.getActorSystem().whenTerminated().isCompleted()).isFalse(); + assertThat(pekkoRpcService.getActorSystem().whenTerminated().isCompleted()).isFalse(); return new RpcServiceShutdownTestHelper( Collections.unmodifiableCollection(onStopFutures), terminationFuture); @@ -378,13 +381,14 @@ public void waitForRpcServiceTermination() throws ExecutionException, Interrupte } @Nonnull - private AkkaRpcService startAkkaRpcService() { - final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - return new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.defaultConfiguration()); + private PekkoRpcService startRpcService() { + final ActorSystem actorSystem = PekkoUtils.createDefaultActorSystem(); + return new PekkoRpcService( + actorSystem, PekkoRpcServiceConfiguration.defaultConfiguration()); } private static class CountingAsynchronousOnStopEndpoint - extends AkkaRpcActorTest.AsynchronousOnStopEndpoint { + extends PekkoRpcActorTest.AsynchronousOnStopEndpoint { private final CountDownLatch countDownLatch; diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaUtilsTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoUtilsTest.java similarity index 52% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaUtilsTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoUtilsTest.java index eff94b36d84d4..fdbcf9e66c78c 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaUtilsTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/PekkoUtilsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -35,226 +35,223 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the {@link AkkaUtils}. */ -class AkkaUtilsTest { +/** Tests for the {@link PekkoUtils}. */ +class PekkoUtilsTest { @Test - void getHostFromAkkaURLForRemoteAkkaURL() throws Exception { + void getHostFromRpcURLForRemoteRpcURL() throws Exception { final String host = "127.0.0.1"; final int port = 1234; final InetSocketAddress address = new InetSocketAddress(host, port); - final String remoteAkkaUrl = - AkkaRpcServiceUtils.getRpcUrl( + final String remoteRpcURL = + PekkoRpcServiceUtils.getRpcUrl( host, port, "actor", AddressResolution.NO_ADDRESS_RESOLUTION, - AkkaRpcServiceUtils.AkkaProtocol.TCP); + PekkoRpcServiceUtils.Protocol.TCP); - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(remoteAkkaUrl); + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(remoteRpcURL); assertThat(result).isEqualTo(address); } @Test - void getHostFromAkkaURLThrowsExceptionIfAddressCannotBeRetrieved() throws Exception { - final String localAkkaURL = "akka://flink/user/actor"; + void getHostFromRpcURLThrowsExceptionIfAddressCannotBeRetrieved() throws Exception { + final String localRpcURL = "pekko://flink/user/actor"; - assertThatThrownBy(() -> AkkaUtils.getInetSocketAddressFromAkkaURL(localAkkaURL)) + assertThatThrownBy(() -> PekkoUtils.getInetSocketAddressFromRpcURL(localRpcURL)) .isInstanceOf(Exception.class); } @Test - void getHostFromAkkaURLReturnsHostAfterAtSign() throws Exception { - final String url = "akka.tcp://flink@localhost:1234/user/jobmanager"; + void getHostFromRpcURLReturnsHostAfterAtSign() throws Exception { + final String url = "pekko.tcp://flink@localhost:1234/user/jobmanager"; final InetSocketAddress expected = new InetSocketAddress("localhost", 1234); - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(expected); } @Test - void getHostFromAkkaURLHandlesAkkaTcpProtocol() throws Exception { - final String url = "akka.tcp://flink@localhost:1234/user/jobmanager"; + void getHostFromRpcURLHandlesAkkaTcpProtocol() throws Exception { + final String url = "pekko.tcp://flink@localhost:1234/user/jobmanager"; final InetSocketAddress expected = new InetSocketAddress("localhost", 1234); - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(expected); } @Test - void getHostFromAkkaURLHandlesAkkaSslTcpProtocol() throws Exception { - final String url = "akka.ssl.tcp://flink@localhost:1234/user/jobmanager"; + void getHostFromRpcURLHandlesAkkaSslTcpProtocol() throws Exception { + final String url = "pekko.ssl.tcp://flink@localhost:1234/user/jobmanager"; final InetSocketAddress expected = new InetSocketAddress("localhost", 1234); - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(expected); } @Test - void getHostFromAkkaURLHandlesIPv4Addresses() throws Exception { + void getHostFromRpcURLHandlesIPv4Addresses() throws Exception { final String ipv4Address = "192.168.0.1"; final int port = 1234; final InetSocketAddress address = new InetSocketAddress(ipv4Address, port); - final String url = "akka://flink@" + ipv4Address + ":" + port + "/user/jobmanager"; - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final String url = "pekko://flink@" + ipv4Address + ":" + port + "/user/jobmanager"; + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(address); } @Test - void getHostFromAkkaURLHandlesIPv6Addresses() throws Exception { + void getHostFromRpcURLHandlesIPv6Addresses() throws Exception { final String ipv6Address = "2001:db8:10:11:12:ff00:42:8329"; final int port = 1234; final InetSocketAddress address = new InetSocketAddress(ipv6Address, port); - final String url = "akka://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final String url = "pekko://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(address); } @Test - void getHostFromAkkaURLHandlesIPv6AddressesTcp() throws Exception { + void getHostFromRpcURLHandlesIPv6AddressesTcp() throws Exception { final String ipv6Address = "2001:db8:10:11:12:ff00:42:8329"; final int port = 1234; final InetSocketAddress address = new InetSocketAddress(ipv6Address, port); - final String url = "akka.tcp://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + final String url = "pekko.tcp://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(address); } @Test - void getHostFromAkkaURLHandlesIPv6AddressesSsl() throws Exception { + void getHostFromRpcURLHandlesIPv6AddressesSsl() throws Exception { final String ipv6Address = "2001:db8:10:11:12:ff00:42:8329"; final int port = 1234; final InetSocketAddress address = new InetSocketAddress(ipv6Address, port); final String url = - "akka.ssl.tcp://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; - final InetSocketAddress result = AkkaUtils.getInetSocketAddressFromAkkaURL(url); + "pekko.ssl.tcp://flink@[" + ipv6Address + "]:" + port + "/user/jobmanager"; + final InetSocketAddress result = PekkoUtils.getInetSocketAddressFromRpcURL(url); assertThat(result).isEqualTo(address); } @Test - void getAkkaConfigNormalizesHostName() { + void getConfigNormalizesHostName() { final Configuration configuration = new Configuration(); final String hostname = "AbC123foOBaR"; final int port = 1234; - final Config akkaConfig = - AkkaUtils.getAkkaConfig(configuration, new HostAndPort(hostname, port)); + final Config config = PekkoUtils.getConfig(configuration, new HostAndPort(hostname, port)); - assertThat(akkaConfig.getString("akka.remote.classic.netty.tcp.hostname")) + assertThat(config.getString("pekko.remote.classic.netty.tcp.hostname")) .isEqualTo(NetUtils.unresolvedHostToNormalizedString(hostname)); } @Test - void getAkkaConfigDefaultsToLocalHost() throws UnknownHostException { - final Config akkaConfig = - AkkaUtils.getAkkaConfig(new Configuration(), new HostAndPort("", 0)); + void getConfigDefaultsToLocalHost() throws UnknownHostException { + final Config config = PekkoUtils.getConfig(new Configuration(), new HostAndPort("", 0)); - final String hostname = akkaConfig.getString("akka.remote.classic.netty.tcp.hostname"); + final String hostname = config.getString("pekko.remote.classic.netty.tcp.hostname"); assertThat(InetAddress.getByName(hostname).isLoopbackAddress()).isTrue(); } @Test - void getAkkaConfigDefaultsToForkJoinExecutor() { - final Config akkaConfig = AkkaUtils.getAkkaConfig(new Configuration(), null); + void getConfigDefaultsToForkJoinExecutor() { + final Config config = PekkoUtils.getConfig(new Configuration(), null); - assertThat(akkaConfig.getString("akka.actor.default-dispatcher.executor")) + assertThat(config.getString("pekko.actor.default-dispatcher.executor")) .isEqualTo("fork-join-executor"); } @Test - void getAkkaConfigSetsExecutorWithThreadPriority() { + void getConfigSetsExecutorWithThreadPriority() { final int threadPriority = 3; final int minThreads = 1; final int maxThreads = 3; - final Config akkaConfig = - AkkaUtils.getAkkaConfig( + final Config config = + PekkoUtils.getConfig( new Configuration(), new HostAndPort("localhost", 1234), null, - AkkaUtils.getThreadPoolExecutorConfig( + PekkoUtils.getThreadPoolExecutorConfig( new RpcSystem.FixedThreadPoolExecutorConfiguration( minThreads, maxThreads, threadPriority))); - assertThat(akkaConfig.getString("akka.actor.default-dispatcher.executor")) + assertThat(config.getString("pekko.actor.default-dispatcher.executor")) .isEqualTo("thread-pool-executor"); - assertThat(akkaConfig.getInt("akka.actor.default-dispatcher.thread-priority")) + assertThat(config.getInt("pekko.actor.default-dispatcher.thread-priority")) .isEqualTo(threadPriority); assertThat( - akkaConfig.getInt( - "akka.actor.default-dispatcher.thread-pool-executor.core-pool-size-min")) + config.getInt( + "pekko.actor.default-dispatcher.thread-pool-executor.core-pool-size-min")) .isEqualTo(minThreads); assertThat( - akkaConfig.getInt( - "akka.actor.default-dispatcher.thread-pool-executor.core-pool-size-max")) + config.getInt( + "pekko.actor.default-dispatcher.thread-pool-executor.core-pool-size-max")) .isEqualTo(maxThreads); } @Test - void getAkkaConfigHandlesIPv6Address() { + void getConfigHandlesIPv6Address() { final String ipv6AddressString = "2001:db8:10:11:12:ff00:42:8329"; - final Config akkaConfig = - AkkaUtils.getAkkaConfig( - new Configuration(), new HostAndPort(ipv6AddressString, 1234)); + final Config config = + PekkoUtils.getConfig(new Configuration(), new HostAndPort(ipv6AddressString, 1234)); - assertThat(akkaConfig.getString("akka.remote.classic.netty.tcp.hostname")) + assertThat(config.getString("pekko.remote.classic.netty.tcp.hostname")) .isEqualTo(NetUtils.unresolvedHostToNormalizedString(ipv6AddressString)); } @Test - void getAkkaConfigDefaultsStartupTimeoutTo10TimesOfAskTimeout() { + void getConfigDefaultsStartupTimeoutTo10TimesOfAskTimeout() { final Configuration configuration = new Configuration(); configuration.set(AkkaOptions.ASK_TIMEOUT_DURATION, Duration.ofMillis(100)); - final Config akkaConfig = - AkkaUtils.getAkkaConfig(configuration, new HostAndPort("localhost", 31337)); + final Config config = + PekkoUtils.getConfig(configuration, new HostAndPort("localhost", 31337)); - assertThat(akkaConfig.getString("akka.remote.startup-timeout")).isEqualTo("1000ms"); + assertThat(config.getString("pekko.remote.startup-timeout")).isEqualTo("1000ms"); } @Test - void getAkkaConfigSslEngineProviderWithoutCertFingerprint() { + void getConfigSslEngineProviderWithoutCertFingerprint() { final Configuration configuration = new Configuration(); configuration.setBoolean(SecurityOptions.SSL_INTERNAL_ENABLED, true); - final Config akkaConfig = - AkkaUtils.getAkkaConfig(configuration, new HostAndPort("localhost", 31337)); - final Config sslConfig = akkaConfig.getConfig("akka.remote.classic.netty.ssl"); + final Config config = + PekkoUtils.getConfig(configuration, new HostAndPort("localhost", 31337)); + final Config sslConfig = config.getConfig("pekko.remote.classic.netty.ssl"); assertThat(sslConfig.getString("ssl-engine-provider")) - .isEqualTo("org.apache.flink.runtime.rpc.akka.CustomSSLEngineProvider"); + .isEqualTo("org.apache.flink.runtime.rpc.pekko.CustomSSLEngineProvider"); assertThat(sslConfig.getStringList("security.cert-fingerprints")).isEmpty(); } @Test - void getAkkaConfigSslEngineProviderWithCertFingerprint() { + void getConfigSslEngineProviderWithCertFingerprint() { final Configuration configuration = new Configuration(); configuration.setBoolean(SecurityOptions.SSL_INTERNAL_ENABLED, true); final String fingerprint = "A8:98:5D:3A:65:E5:E5:C4:B2:D7:D6:6D:40:C6:DD:2F:B1:9C:54:36"; configuration.setString(SecurityOptions.SSL_INTERNAL_CERT_FINGERPRINT, fingerprint); - final Config akkaConfig = - AkkaUtils.getAkkaConfig(configuration, new HostAndPort("localhost", 31337)); - final Config sslConfig = akkaConfig.getConfig("akka.remote.classic.netty.ssl"); + final Config config = + PekkoUtils.getConfig(configuration, new HostAndPort("localhost", 31337)); + final Config sslConfig = config.getConfig("pekko.remote.classic.netty.ssl"); assertThat(sslConfig.getString("ssl-engine-provider")) - .isEqualTo("org.apache.flink.runtime.rpc.akka.CustomSSLEngineProvider"); + .isEqualTo("org.apache.flink.runtime.rpc.pekko.CustomSSLEngineProvider"); assertThat(sslConfig.getStringList("security.cert-fingerprints")).contains(fingerprint); } } diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RemoteAkkaRpcActorTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RemotePekkoRpcActorTest.java similarity index 67% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RemoteAkkaRpcActorTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RemotePekkoRpcActorTest.java index 255ebf126d21c..62d3c2f63498d 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RemoteAkkaRpcActorTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RemotePekkoRpcActorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.FlinkAssertions; @@ -36,22 +36,22 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for remote AkkaRpcActors. */ -class RemoteAkkaRpcActorTest { +/** Tests for remote {@link PekkoRpcActor}s. */ +class RemotePekkoRpcActorTest { - private static AkkaRpcService rpcService; - private static AkkaRpcService otherRpcService; + private static PekkoRpcService rpcService; + private static PekkoRpcService otherRpcService; private static final Configuration configuration = new Configuration(); @BeforeAll static void setupClass() throws Exception { rpcService = - AkkaRpcServiceUtils.createRemoteRpcService( + PekkoRpcServiceUtils.createRemoteRpcService( configuration, "localhost", "0", null, Optional.empty()); otherRpcService = - AkkaRpcServiceUtils.createRemoteRpcService( + PekkoRpcServiceUtils.createRemoteRpcService( configuration, "localhost", "0", null, Optional.empty()); } @@ -62,15 +62,15 @@ static void teardownClass() throws InterruptedException, ExecutionException, Tim @Test void canRespondWithNullValueRemotely() throws Exception { - try (final AkkaRpcActorTest.NullRespondingEndpoint nullRespondingEndpoint = - new AkkaRpcActorTest.NullRespondingEndpoint(rpcService)) { + try (final PekkoRpcActorTest.NullRespondingEndpoint nullRespondingEndpoint = + new PekkoRpcActorTest.NullRespondingEndpoint(rpcService)) { nullRespondingEndpoint.start(); - final AkkaRpcActorTest.NullRespondingGateway rpcGateway = + final PekkoRpcActorTest.NullRespondingGateway rpcGateway = otherRpcService .connect( nullRespondingEndpoint.getAddress(), - AkkaRpcActorTest.NullRespondingGateway.class) + PekkoRpcActorTest.NullRespondingGateway.class) .join(); final CompletableFuture nullValuedResponseFuture = rpcGateway.foobar(); @@ -81,15 +81,15 @@ void canRespondWithNullValueRemotely() throws Exception { @Test void canRespondWithSynchronousNullValueRemotely() throws Exception { - try (final AkkaRpcActorTest.NullRespondingEndpoint nullRespondingEndpoint = - new AkkaRpcActorTest.NullRespondingEndpoint(rpcService)) { + try (final PekkoRpcActorTest.NullRespondingEndpoint nullRespondingEndpoint = + new PekkoRpcActorTest.NullRespondingEndpoint(rpcService)) { nullRespondingEndpoint.start(); - final AkkaRpcActorTest.NullRespondingGateway rpcGateway = + final PekkoRpcActorTest.NullRespondingGateway rpcGateway = otherRpcService .connect( nullRespondingEndpoint.getAddress(), - AkkaRpcActorTest.NullRespondingGateway.class) + PekkoRpcActorTest.NullRespondingGateway.class) .join(); final Integer value = rpcGateway.synchronousFoobar(); @@ -100,39 +100,41 @@ void canRespondWithSynchronousNullValueRemotely() throws Exception { @Test void canRespondWithSerializedValueRemotely() throws Exception { - try (final AkkaRpcActorTest.SerializedValueRespondingEndpoint endpoint = - new AkkaRpcActorTest.SerializedValueRespondingEndpoint(rpcService)) { + try (final PekkoRpcActorTest.SerializedValueRespondingEndpoint endpoint = + new PekkoRpcActorTest.SerializedValueRespondingEndpoint(rpcService)) { endpoint.start(); - final AkkaRpcActorTest.SerializedValueRespondingGateway remoteGateway = + final PekkoRpcActorTest.SerializedValueRespondingGateway remoteGateway = otherRpcService .connect( endpoint.getAddress(), - AkkaRpcActorTest.SerializedValueRespondingGateway.class) + PekkoRpcActorTest.SerializedValueRespondingGateway.class) .join(); assertThat(remoteGateway.getSerializedValueSynchronously()) - .isEqualTo(AkkaRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE); + .isEqualTo( + PekkoRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE); final CompletableFuture> responseFuture = remoteGateway.getSerializedValue(); assertThat(responseFuture.get()) - .isEqualTo(AkkaRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE); + .isEqualTo( + PekkoRpcActorTest.SerializedValueRespondingEndpoint.SERIALIZED_VALUE); } } @Test void failsRpcResultImmediatelyIfEndpointIsStopped() throws Exception { - try (final AkkaRpcActorTest.SerializedValueRespondingEndpoint endpoint = - new AkkaRpcActorTest.SerializedValueRespondingEndpoint(rpcService)) { + try (final PekkoRpcActorTest.SerializedValueRespondingEndpoint endpoint = + new PekkoRpcActorTest.SerializedValueRespondingEndpoint(rpcService)) { endpoint.start(); - final AkkaRpcActorTest.SerializedValueRespondingGateway gateway = + final PekkoRpcActorTest.SerializedValueRespondingGateway gateway = otherRpcService .connect( endpoint.getAddress(), - AkkaRpcActorTest.SerializedValueRespondingGateway.class) + PekkoRpcActorTest.SerializedValueRespondingGateway.class) .join(); endpoint.close(); @@ -146,18 +148,18 @@ void failsRpcResultImmediatelyIfEndpointIsStopped() throws Exception { @Test void failsRpcResultImmediatelyIfRemoteRpcServiceIsNotAvailable() throws Exception { - final AkkaRpcService toBeClosedRpcService = - AkkaRpcServiceUtils.createRemoteRpcService( + final PekkoRpcService toBeClosedRpcService = + PekkoRpcServiceUtils.createRemoteRpcService( configuration, "localhost", "0", null, Optional.empty()); - try (final AkkaRpcActorTest.SerializedValueRespondingEndpoint endpoint = - new AkkaRpcActorTest.SerializedValueRespondingEndpoint(toBeClosedRpcService)) { + try (final PekkoRpcActorTest.SerializedValueRespondingEndpoint endpoint = + new PekkoRpcActorTest.SerializedValueRespondingEndpoint(toBeClosedRpcService)) { endpoint.start(); - final AkkaRpcActorTest.SerializedValueRespondingGateway gateway = + final PekkoRpcActorTest.SerializedValueRespondingGateway gateway = otherRpcService .connect( endpoint.getAddress(), - AkkaRpcActorTest.SerializedValueRespondingGateway.class) + PekkoRpcActorTest.SerializedValueRespondingGateway.class) .join(); toBeClosedRpcService.closeAsync().join(); diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RobustActorSystemTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystemTest.java similarity index 94% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RobustActorSystemTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystemTest.java index 8bddf57a34c07..5239200f7f3b3 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/RobustActorSystemTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RobustActorSystemTest.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.concurrent.TestingUncaughtExceptionHandler; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.Props; -import akka.japi.pf.ReceiveBuilder; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.Props; +import org.apache.pekko.japi.pf.ReceiveBuilder; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -44,7 +44,7 @@ void setup() { robustActorSystem = RobustActorSystem.create( "testSystem", - AkkaUtils.getForkJoinExecutorConfig( + PekkoUtils.getForkJoinExecutorConfig( RpcUtils.getTestForkJoinExecutorConfiguration()), testingUncaughtExceptionHandler); } diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValueTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValueTest.java similarity index 83% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValueTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValueTest.java index 3bc1d357d6993..1302f57e5cf8e 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcSerializedValueTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/RpcSerializedValueTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.util.InstantiationUtil; @@ -31,21 +31,21 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for the {@link AkkaRpcSerializedValue}. */ -class AkkaRpcSerializedValueTest { +/** Tests for the {@link RpcSerializedValue}. */ +class RpcSerializedValueTest { @Test void testNullValue() throws Exception { - AkkaRpcSerializedValue serializedValue = AkkaRpcSerializedValue.valueOf(null); + RpcSerializedValue serializedValue = RpcSerializedValue.valueOf(null); assertThat(serializedValue.getSerializedData()).isNull(); assertThat(serializedValue.getSerializedDataLength()).isEqualTo(0); assertThat((Object) serializedValue.deserializeValue(getClass().getClassLoader())).isNull(); - AkkaRpcSerializedValue otherSerializedValue = AkkaRpcSerializedValue.valueOf(null); + RpcSerializedValue otherSerializedValue = RpcSerializedValue.valueOf(null); assertThat(otherSerializedValue).isEqualTo(serializedValue); assertThat(otherSerializedValue.hashCode()).isEqualTo(serializedValue.hashCode()); - AkkaRpcSerializedValue clonedSerializedValue = InstantiationUtil.clone(serializedValue); + RpcSerializedValue clonedSerializedValue = InstantiationUtil.clone(serializedValue); assertThat(clonedSerializedValue.getSerializedData()).isNull(); assertThat(clonedSerializedValue.getSerializedDataLength()).isEqualTo(0); assertThat((Object) clonedSerializedValue.deserializeValue(getClass().getClassLoader())) @@ -73,17 +73,17 @@ static Stream serializationArguments() { @ParameterizedTest @MethodSource("serializationArguments") void testNotNullValues(Object value) throws Exception { - AkkaRpcSerializedValue serializedValue = AkkaRpcSerializedValue.valueOf(value); + RpcSerializedValue serializedValue = RpcSerializedValue.valueOf(value); assertThat(serializedValue.getSerializedData()).isNotNull(); assertThat(serializedValue.getSerializedDataLength()).isGreaterThan(0); assertThat((Object) serializedValue.deserializeValue(getClass().getClassLoader())) .isEqualTo(value); - AkkaRpcSerializedValue otherSerializedValue = AkkaRpcSerializedValue.valueOf(value); + RpcSerializedValue otherSerializedValue = RpcSerializedValue.valueOf(value); assertThat(otherSerializedValue).isEqualTo(serializedValue); assertThat(otherSerializedValue.hashCode()).isEqualTo(serializedValue.hashCode()); - AkkaRpcSerializedValue clonedSerializedValue = InstantiationUtil.clone(serializedValue); + RpcSerializedValue clonedSerializedValue = InstantiationUtil.clone(serializedValue); assertThat(clonedSerializedValue.getSerializedData()) .isEqualTo(serializedValue.getSerializedData()); assertThat((Object) clonedSerializedValue.deserializeValue(getClass().getClassLoader())) diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/SupervisorActorTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/SupervisorActorTest.java similarity index 92% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/SupervisorActorTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/SupervisorActorTest.java index 9b2bbce3c9c0d..07b88b42d59c4 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/SupervisorActorTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/SupervisorActorTest.java @@ -16,17 +16,17 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; -import akka.actor.AbstractActor; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Props; -import akka.actor.Terminated; -import akka.japi.pf.ReceiveBuilder; +import org.apache.pekko.actor.AbstractActor; +import org.apache.pekko.actor.ActorRef; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Props; +import org.apache.pekko.actor.Terminated; +import org.apache.pekko.japi.pf.ReceiveBuilder; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -53,7 +53,7 @@ void completesTerminationFutureIfActorStops() { SupervisorActor.startSupervisorActor(actorSystem, actorSystem.getDispatcher()); final SupervisorActor.ActorRegistration actorRegistration = - startAkkaRpcActor(supervisor, "foobar"); + startRpcActor(supervisor, "foobar"); final CompletableFuture terminationFuture = actorRegistration.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); @@ -73,7 +73,7 @@ void completesTerminationFutureExceptionallyIfActorStopsExceptionally() throws E SupervisorActor.startSupervisorActor(actorSystem, actorSystem.getDispatcher()); final SupervisorActor.ActorRegistration actorRegistration = - startAkkaRpcActor(supervisor, "foobar"); + startRpcActor(supervisor, "foobar"); final CompletableFuture terminationFuture = actorRegistration.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); @@ -101,7 +101,7 @@ void completesTerminationFutureExceptionallyIfActorStopsWithoutReason() SupervisorActor.startSupervisorActor(actorSystem, actorSystem.getDispatcher()); final SupervisorActor.ActorRegistration actorRegistration = - startAkkaRpcActor(supervisor, "foobar"); + startRpcActor(supervisor, "foobar"); final CompletableFuture terminationFuture = actorRegistration.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); @@ -123,7 +123,7 @@ void completesTerminationFutureExceptionallyIfActorFails() throws Exception { SupervisorActor.startSupervisorActor(actorSystem, actorSystem.getDispatcher()); final SupervisorActor.ActorRegistration actorRegistration = - startAkkaRpcActor(supervisor, "foobar"); + startRpcActor(supervisor, "foobar"); final CompletableFuture terminationFuture = actorRegistration.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); @@ -154,9 +154,9 @@ void completesTerminationFutureOfSiblingsIfActorFails() throws Exception { SupervisorActor.startSupervisorActor(actorSystem, actorSystem.getDispatcher()); final SupervisorActor.ActorRegistration actorRegistration1 = - startAkkaRpcActor(supervisor, "foobar1"); + startRpcActor(supervisor, "foobar1"); final SupervisorActor.ActorRegistration actorRegistration2 = - startAkkaRpcActor(supervisor, "foobar2"); + startRpcActor(supervisor, "foobar2"); final CompletableFuture terminationFuture = actorRegistration2.getTerminationFuture(); assertThat(terminationFuture).isNotDone(); @@ -171,10 +171,10 @@ void completesTerminationFutureOfSiblingsIfActorFails() throws Exception { } } - private SupervisorActor.ActorRegistration startAkkaRpcActor( + private SupervisorActor.ActorRegistration startRpcActor( ActorRef supervisor, String endpointId) { - final SupervisorActor.StartAkkaRpcActorResponse startResponse = - SupervisorActor.startAkkaRpcActor( + final SupervisorActor.StartRpcActorResponse startResponse = + SupervisorActor.startRpcActor( supervisor, terminationFuture -> Props.create(SimpleActor.class, terminationFuture), endpointId); diff --git a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/TimeoutCallStackTest.java b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/TimeoutCallStackTest.java similarity index 91% rename from flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/TimeoutCallStackTest.java rename to flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/TimeoutCallStackTest.java index 77906759c2a4e..718812fa8dbcf 100644 --- a/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/akka/TimeoutCallStackTest.java +++ b/flink-rpc/flink-rpc-akka/src/test/java/org/apache/flink/runtime/rpc/pekko/TimeoutCallStackTest.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc.pekko; import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils; +import org.apache.flink.runtime.concurrent.pekko.ScalaFutureUtils; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; @@ -27,8 +27,8 @@ import org.apache.flink.util.IOUtils; import org.apache.flink.util.concurrent.FutureUtils; -import akka.actor.ActorSystem; -import akka.actor.Terminated; +import org.apache.pekko.actor.ActorSystem; +import org.apache.pekko.actor.Terminated; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -56,9 +56,10 @@ class TimeoutCallStackTest { @BeforeAll static void setup() { - actorSystem = AkkaUtils.createDefaultActorSystem(); + actorSystem = PekkoUtils.createDefaultActorSystem(); rpcService = - new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.defaultConfiguration()); + new PekkoRpcService( + actorSystem, PekkoRpcServiceConfiguration.defaultConfiguration()); } @AfterAll @@ -66,7 +67,7 @@ static void teardown() throws Exception { final CompletableFuture rpcTerminationFuture = rpcService.closeAsync(); final CompletableFuture actorSystemTerminationFuture = - AkkaFutureUtils.toJava(actorSystem.terminate()); + ScalaFutureUtils.toJava(actorSystem.terminate()); FutureUtils.waitForAll(Arrays.asList(rpcTerminationFuture, actorSystemTerminationFuture)) .get(10_000, TimeUnit.MILLISECONDS); diff --git a/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java b/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java index 97f7ec8739224..56b1a81881b6a 100644 --- a/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java +++ b/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java @@ -41,9 +41,10 @@ public class RpcUtils { /** - * HACK: Set to 21474835 seconds, Akka's maximum delay (Akka 2.4.20). The value cannot be - * higher or an {@link IllegalArgumentException} will be thrown during an RPC. Check the private - * method {@code checkMaxDelay()} in {@link akka.actor.LightArrayRevolverScheduler}. + * HACK: Set to 21474835 seconds, Pekko's maximum delay (Akka 2.4.20). The value cannot + * be higher or an {@link IllegalArgumentException} will be thrown during an RPC. Check the + * private method {@code checkMaxDelay()} in {@link + * org.apache.pekko.actor.LightArrayRevolverScheduler}. */ public static final Time INF_TIMEOUT = Time.seconds(21474835); diff --git a/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java b/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java index 685ab5627488d..d8845d04226cd 100644 --- a/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java +++ b/flink-rpc/flink-rpc-core/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java @@ -106,7 +106,7 @@ public String toString() { private void writeObject(ObjectOutputStream oos) throws IOException { // Translate it to byte array so that we can deserialize classes which cannot be found in - // akka class loader. + // pekko class loader. byte[] bytes = InstantiationUtil.serializeObject(methodInvocation); oos.writeObject(bytes); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageNettyServiceImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageNettyServiceImpl.java index 6e1320e1ff1b5..ab85e05beb686 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageNettyServiceImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageNettyServiceImpl.java @@ -140,7 +140,7 @@ public ResultSubpartitionView createResultSubpartitionView( } /** - * Set up input channels in {@link SingleInputGate}. The method will be invoked by the akka rpc + * Set up input channels in {@link SingleInputGate}. The method will be invoked by the pekko rpc * thread at first, and then the method {@link * TieredStorageNettyService#registerConsumer(TieredStoragePartitionId, * TieredStorageSubpartitionId)} will be invoked by the same thread sequentially, which ensures diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index 9b2b2bddc48fa..ad6b7d3246a43 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -89,7 +89,7 @@ private UnmodifiableConfiguration generateConfiguration(final Configuration conf modifiedConfig.set(ClusterOptions.CLUSTER_IO_EXECUTOR_POOL_SIZE, DEFAULT_IO_POOL_SIZE); } - // increase the akka.ask.timeout if not set in order to harden tests on slow CI + // increase the ask.timeout if not set in order to harden tests on slow CI if (!modifiedConfig.contains(AkkaOptions.ASK_TIMEOUT_DURATION)) { modifiedConfig.set(AkkaOptions.ASK_TIMEOUT_DURATION, Duration.ofMinutes(5L)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index 9f61caedbe072..a749115d92cbd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -381,7 +381,7 @@ private enum LeaderRetrievalState { private final Object retrievalLock = new Object(); - private String akkaURL; + private String rpcURL; private LeaderRetrievalState retrievalState = LeaderRetrievalState.NOT_RETRIEVED; private Exception exception; @@ -416,12 +416,12 @@ public InetAddress findConnectingAddress(Duration timeout, Duration startLogging + "while waiting for the leader retrieval."); } } else if (retrievalState == LeaderRetrievalState.NEWLY_RETRIEVED) { - targetAddress = rpcSystemUtils.getInetSocketAddressFromRpcUrl(akkaURL); + targetAddress = rpcSystemUtils.getInetSocketAddressFromRpcUrl(rpcURL); LOG.debug( - "Retrieved new target address {} for akka URL {}.", + "Retrieved new target address {} for RPC URL {}.", targetAddress, - akkaURL); + rpcURL); retrievalState = LeaderRetrievalState.RETRIEVED; @@ -509,8 +509,8 @@ public InetAddress findConnectingAddress(Duration timeout, Duration startLogging } catch (Exception e) { throw new LeaderRetrievalException( "Could not retrieve the connecting address to the " - + "current leader with the akka URL " - + akkaURL + + "current leader with the pekko URL " + + rpcURL + ".", e); } @@ -520,7 +520,7 @@ public InetAddress findConnectingAddress(Duration timeout, Duration startLogging public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { if (leaderAddress != null && !leaderAddress.isEmpty()) { synchronized (retrievalLock) { - akkaURL = leaderAddress; + rpcURL = leaderAddress; retrievalState = LeaderRetrievalState.NEWLY_RETRIEVED; retrievalLock.notifyAll(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java index 691086e1195d3..8ccb1d2f95904 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java @@ -40,13 +40,13 @@ public class LeaderRetrievalUtils { private static final Logger LOG = LoggerFactory.getLogger(LeaderRetrievalUtils.class); /** - * Retrieves the leader akka url and the current leader session ID. The values are stored in a + * Retrieves the leader pekko url and the current leader session ID. The values are stored in a * {@link LeaderInformation} instance. * * @param leaderRetrievalService Leader retrieval service to retrieve the leader connection * information * @param timeout Timeout when to give up looking for the leader - * @return LeaderInformation containing the leader's akka URL and the current leader session ID + * @return LeaderInformation containing the leader's rpc URL and the current leader session ID * @throws LeaderRetrievalException */ public static LeaderInformation retrieveLeaderInformation( @@ -110,7 +110,7 @@ public static InetAddress findConnectingAddress( /** * Helper class which is used by the retrieveLeaderInformation method to retrieve the leader's - * akka URL and the current leader session ID. + * rpc URL and the current leader session ID. */ public static class LeaderInformationListener implements LeaderRetrievalListener { private final CompletableFuture connectionInfoFuture = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 7153f320ef9e8..f6c4a59ca5a01 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -255,12 +255,12 @@ public WebMonitorEndpoint( } private VertexThreadInfoTracker initializeThreadInfoTracker(ScheduledExecutorService executor) { - final Duration akkaTimeout = clusterConfiguration.get(AkkaOptions.ASK_TIMEOUT_DURATION); + final Duration askTimeout = clusterConfiguration.get(AkkaOptions.ASK_TIMEOUT_DURATION); final Duration flameGraphCleanUpInterval = clusterConfiguration.get(RestOptions.FLAMEGRAPH_CLEANUP_INTERVAL); final ThreadInfoRequestCoordinator threadInfoRequestCoordinator = - new ThreadInfoRequestCoordinator(executor, akkaTimeout); + new ThreadInfoRequestCoordinator(executor, askTimeout); return VertexThreadInfoTrackerBuilder.newBuilder( resourceManagerRetriever, executor, restConfiguration.getTimeout()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 42398ba446975..4281a7f344d66 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -63,7 +63,7 @@ public class PartialConsumePipelinedResultTest extends TestLogger { private static Configuration getFlinkConfiguration() { final Configuration config = new Configuration(); - config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT); + config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_ASK_TIMEOUT); config.setInteger( NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, NUMBER_OF_NETWORK_BUFFERS); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index 65f5637b2fdf6..e5e4c896b56fe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -68,7 +68,7 @@ public class SlotCountExceedingParallelismTest extends TestLogger { private static Configuration getFlinkConfiguration() { final Configuration config = new Configuration(); - config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT); + config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_ASK_TIMEOUT); return config; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java index acf1a034a82ae..e462a0d23c501 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java @@ -80,7 +80,7 @@ public class TestingJobMasterGatewayBuilder { private static final ResourceID RESOURCE_MANAGER_ID = ResourceID.generate(); private static final JobMasterId JOB_MASTER_ID = JobMasterId.generate(); - private String address = "akka.tcp://flink@localhost:6130/user/jobmanager"; + private String address = "pekko.tcp://flink@localhost:6130/user/jobmanager"; private String hostname = "localhost"; private Supplier> cancelFunction = () -> CompletableFuture.completedFuture(Acknowledge.get()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java index 30cfd6ad7dcaf..de8e782254940 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java @@ -32,7 +32,7 @@ class StandaloneLeaderElectionTest { private static final UUID SESSION_ID = UUID.randomUUID(); - private static final String TEST_URL = "akka://users/jobmanager"; + private static final String TEST_URL = "pekko://users/jobmanager"; /** * Tests that the standalone leader election and retrieval service return the same leader URL. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java index e6dbac26ac60c..28cd7276663f5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java @@ -97,7 +97,7 @@ class ZooKeeperLeaderElectionTest { private Configuration configuration; private static final String COMPONENT_ID = "component-id"; - private static final String LEADER_ADDRESS = "akka//user/jobmanager"; + private static final String LEADER_ADDRESS = "pekko://user/jobmanager"; private static final long timeout = 200L * 1000L; private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionTest.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java index e4577951e9115..e1d2c650a3bf1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java @@ -32,7 +32,7 @@ /** Tests for {@link DefaultLeaderElectionService}. */ class DefaultLeaderRetrievalServiceTest { - private static final String TEST_URL = "akka//user/jobmanager"; + private static final String TEST_URL = "pekko://user/jobmanager"; @Test void testNotifyLeaderAddress() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java index 9f641522426cf..f5854c1b83de5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcSSLAuthITCase.java @@ -86,7 +86,7 @@ public void testConnectFailure() throws Exception { // to test whether the test is still good: // - create actorSystem2 with sslConfig1 (same as actorSystem1) and see that both can // connect - // - set 'require-mutual-authentication = off' in the AkkaUtils ssl config section + // - set 'require-mutual-authentication = off' in the ConfigUtils ssl config section rpcService1 = RpcSystem.load() .localServiceBuilder(sslConfig1) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index 0e6e781b3ffbf..72c036332773d 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -751,7 +751,7 @@ object ExecutionEnvironment { /** * Creates a remote execution environment. The remote environment sends (parts of) the program to * a cluster for execution. Note that all file paths used in the program must be accessible from - * the cluster. The custom configuration file is used to configure Akka specific configuration + * the cluster. The custom configuration file is used to configure Pekko specific configuration * parameters for the Client only; Program parallelism can be set via * [[ExecutionEnvironment.setParallelism]]. * diff --git a/flink-table/flink-table-planner-loader/pom.xml b/flink-table/flink-table-planner-loader/pom.xml index 6b86a603735a0..6dd0b68a6d9f8 100644 --- a/flink-table/flink-table-planner-loader/pom.xml +++ b/flink-table/flink-table-planner-loader/pom.xml @@ -61,7 +61,7 @@ true - + * * diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecDynamicFilteringDataCollector.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecDynamicFilteringDataCollector.java index 1fe31a0d13d69..f38181badfcc7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecDynamicFilteringDataCollector.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecDynamicFilteringDataCollector.java @@ -60,7 +60,7 @@ public class BatchExecDynamicFilteringDataCollector extends ExecNodeBase .withDescription( "If the collector collects more data than the threshold (default is 8M), " + "an empty DynamicFilterEvent with a flag only will be sent to Coordinator, " - + "which could avoid exceeding the akka limit and out-of-memory (see " + + "which could avoid exceeding the pekko limit and out-of-memory (see " + AkkaOptions.FRAMESIZE.key() + "). Otherwise a DynamicFilterEvent with all deduplicated records will be sent to Coordinator."); diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/container/FlinkContainersSettings.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/container/FlinkContainersSettings.java index fc531a1cf5742..0f8c712b0916a 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/container/FlinkContainersSettings.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/container/FlinkContainersSettings.java @@ -391,7 +391,7 @@ private Properties defaultLoggingProperties() { logProperties.setProperty("appender.rolling.fileName", "${sys:log.file}"); logProperties.setProperty("appender.rolling.filePattern", "${sys:log.file}.%i"); logProperties.setProperty("rootLogger.appenderRef.rolling.ref", "RollingFileAppender"); - logProperties.setProperty("logger.akka.name", "akka"); + logProperties.setProperty("logger.pekko.name", "org.apache.pekko"); logProperties.setProperty("appender.console.type", "CONSOLE"); logProperties.setProperty("appender.rolling.append", "true"); logProperties.setProperty("appender.console.layout.type", "PatternLayout"); @@ -409,7 +409,7 @@ private Properties defaultLoggingProperties() { logProperties.setProperty("logger.kafka.name", "org.apache.kafka"); logProperties.setProperty("logger.netty.level", "OFF"); logProperties.setProperty("appender.rolling.type", "RollingFile"); - logProperties.setProperty("logger.akka.level", "INFO"); + logProperties.setProperty("logger.pekko.level", "INFO"); logProperties.setProperty("logger.hadoop.level", "INFO"); logProperties.setProperty( "appender.rolling.layout.pattern", diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index ab912a62dd331..1b0f2995c60c5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -101,7 +101,7 @@ public class AccumulatorLiveITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT); + config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_ASK_TIMEOUT); config.setLong(HeartbeatManagerOptions.HEARTBEAT_INTERVAL, HEARTBEAT_INTERVAL); return config; diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index ebed95b9e1477..c0899dcfdcf5c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -85,7 +85,7 @@ private static Configuration getConfiguration() { verifyJvmOptions(); Configuration config = new Configuration(); config.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); - config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT); + config.set(AkkaOptions.ASK_TIMEOUT_DURATION, TestingUtils.DEFAULT_ASK_TIMEOUT); config.set(TaskManagerOptions.MEMORY_SEGMENT_SIZE, MemorySize.parse("4096")); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 2048); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java index 49b7427020907..aea53b147fef5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java @@ -67,7 +67,7 @@ private Configuration getConfiguration() { final Inet6Address ipv6address = getLocalIPv6Address(); if (ipv6address == null) { throw new AssumptionViolatedException( - "--- Cannot find a non-loopback local IPv6 address that Akka/Netty can bind to; skipping IPv6HostnamesITCase"); + "--- Cannot find a non-loopback local IPv6 address that Pekko/Netty can bind to; skipping IPv6HostnamesITCase"); } final String addressString = ipv6address.getHostAddress(); log.info("Test will use IPv6 address " + addressString + " for connection tests"); @@ -143,8 +143,8 @@ private Inet6Address getLocalIPv6Address() { sock.bind(socketAddress); sock.close(); - // test whether Akka's netty can bind to the address - log.info("Testing whether Akka can use " + addr); + // test whether Pekko's netty can bind to the address + log.info("Testing whether Pekko can use " + addr); final RpcService rpcService = RpcSystem.load() // this port is only used for advertising (==no port diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 648cb8510e773..85c1c3e0da5c7 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -136,16 +136,16 @@ public abstract class YarnTestBase { // occurs if a TM disconnects from a JM because it is no longer hosting any slots Pattern.compile("has no more allocated slots for job"), // can happen if another process hasn't fully started yet - Pattern.compile("akka.actor.ActorNotFound: Actor not found for"), + Pattern.compile("org.apache.pekko.actor.ActorNotFound: Actor not found for"), // can happen if another process hasn't fully started yet Pattern.compile("RpcConnectionException: Could not connect to rpc endpoint under address"), // rest handler whose registration is logged on DEBUG level Pattern.compile("JobExceptionsHandler"), - Pattern.compile("akka\\.remote\\.RemoteTransportExceptionNoStackTrace"), + Pattern.compile("org\\.apache\\.pekko\\.remote\\.RemoteTransportExceptionNoStackTrace"), // workaround for annoying InterruptedException logging: // https://issues.apache.org/jira/browse/YARN-1022 Pattern.compile("java\\.lang\\.InterruptedException"), - // very specific on purpose; whitelist meaningless exceptions that occur during akka + // very specific on purpose; whitelist meaningless exceptions that occur during Pekko // shutdown: Pattern.compile( "Remote connection to \\[.*\\] failed with java.net.ConnectException: Connection refused"), @@ -153,12 +153,12 @@ public abstract class YarnTestBase { "Remote connection to \\[.*\\] failed with java.nio.channels.NotYetConnectedException"), Pattern.compile("java\\.io\\.IOException: Connection reset by peer"), Pattern.compile( - "Association with remote system \\[akka.tcp://flink@[^]]+\\] has failed, address is now gated for \\[50\\] ms. Reason: \\[Association failed with \\[akka.tcp://flink@[^]]+\\]\\] Caused by: \\[java.net.ConnectException: Connection refused: [^]]+\\]"), + "Association with remote system \\[pekko.tcp://flink@[^]]+\\] has failed, address is now gated for \\[50\\] ms. Reason: \\[Association failed with \\[pekko.tcp://flink@[^]]+\\]\\] Caused by: \\[java.net.ConnectException: Connection refused: [^]]+\\]"), // filter out expected ResourceManagerException caused by intended shutdown request Pattern.compile(YarnResourceManagerDriver.ERROR_MESSAGE_ON_SHUTDOWN_REQUEST), - // this can happen in Akka on shutdown. + // this can happen in Pekko on shutdown. Pattern.compile( "java\\.util\\.concurrent\\.RejectedExecutionException: Worker has already been shutdown"), Pattern.compile("org\\.apache\\.flink.util\\.FlinkException: Stopping JobMaster"), @@ -166,9 +166,9 @@ public abstract class YarnTestBase { "org\\.apache\\.flink.util\\.FlinkException: JobManager is shutting down\\."), Pattern.compile("lost the leadership."), Pattern.compile( - "akka.remote.transport.netty.NettyTransport.*Remote connection to \\[[^]]+\\] failed with java.io.IOException: Broken pipe"), + "org.apache.pekko.remote.transport.netty.NettyTransport.*Remote connection to \\[[^]]+\\] failed with java.io.IOException: Broken pipe"), Pattern.compile( - "akka.remote.transport.netty.NettyTransport.*Remote connection to \\[.+\\] failed with java.net.SocketException: Connection reset"), + "org.apache.pekko.remote.transport.netty.NettyTransport.*Remote connection to \\[.+\\] failed with java.net.SocketException: Connection reset"), // this can happen during cluster shutdown, if AMRMClient happens to be heartbeating Pattern.compile("Exception on heartbeat"), diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBaseTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBaseTest.java index 045610b1f6129..131dd44804f21 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBaseTest.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBaseTest.java @@ -31,9 +31,9 @@ class YarnTestBaseTest { void ensureWhitelistEntryMatches() { ensureWhitelistEntryMatch("465 java.lang.InterruptedException: sleep interrupted"); ensureWhitelistEntryMatch( - "2020-09-19 22:06:19,458 WARN akka.remote.ReliableDeliverySupervisor [] - Association with remote system [akka.tcp://flink@e466f3e261f3:42352] has failed, address is now gated for [50] ms. Reason: [Association failed with [akka.tcp://flink@e466f3e261f3:42352]] Caused by: [java.net.ConnectException: Connection refused: e466f3e261f3/192.168.224.2:42352]"); + "2020-09-19 22:06:19,458 WARN org.apache.pekko.remote.ReliableDeliverySupervisor [] - Association with remote system [pekko.tcp://flink@e466f3e261f3:42352] has failed, address is now gated for [50] ms. Reason: [Association failed with [pekko.tcp://flink@e466f3e261f3:42352]] Caused by: [java.net.ConnectException: Connection refused: e466f3e261f3/192.168.224.2:42352]"); ensureWhitelistEntryMatch( - "2020-10-15 10:31:09,661 WARN akka.remote.transport.netty.NettyTransport [] - Remote connection to [61b81e62b514/192.168.128.2:39365] failed with java.io.IOException: Broken pipe"); + "2020-10-15 10:31:09,661 WARN org.apache.pekko.remote.transport.netty.NettyTransport [] - Remote connection to [61b81e62b514/192.168.128.2:39365] failed with java.io.IOException: Broken pipe"); } private void ensureWhitelistEntryMatch(String probe) { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManagerDriver.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManagerDriver.java index 1251007a60c72..73d6013a0f8a4 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManagerDriver.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManagerDriver.java @@ -83,7 +83,7 @@ public class YarnResourceManagerDriver extends AbstractResourceManagerDriver