Skip to content

Commit ae85de7

Browse files
committed
[hotfix] Remove akka dependency in MetricUtilsTest
1 parent e2d36ba commit ae85de7

File tree

1 file changed

+1
-11
lines changed

1 file changed

+1
-11
lines changed

flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java

+1-11
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,6 @@
2828
import org.apache.flink.runtime.metrics.MetricNames;
2929
import org.apache.flink.runtime.rpc.RpcService;
3030
import org.apache.flink.runtime.rpc.RpcSystem;
31-
import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
3231
import org.apache.flink.runtime.taskexecutor.TaskManagerServices;
3332
import org.apache.flink.runtime.taskexecutor.TaskManagerServicesBuilder;
3433
import org.apache.flink.runtime.taskexecutor.slot.TestingTaskSlotTable;
@@ -40,7 +39,6 @@
4039

4140
import org.apache.flink.shaded.guava18.com.google.common.collect.Sets;
4241

43-
import akka.actor.ActorSystem;
4442
import org.junit.After;
4543
import org.junit.Assert;
4644
import org.junit.Assume;
@@ -54,7 +52,6 @@
5452
import static org.apache.flink.runtime.metrics.util.MetricUtils.METRIC_GROUP_FLINK;
5553
import static org.apache.flink.runtime.metrics.util.MetricUtils.METRIC_GROUP_MANAGED_MEMORY;
5654
import static org.apache.flink.runtime.metrics.util.MetricUtils.METRIC_GROUP_MEMORY;
57-
import static org.hamcrest.CoreMatchers.instanceOf;
5855
import static org.hamcrest.Matchers.is;
5956
import static org.junit.Assert.assertThat;
6057

@@ -83,17 +80,10 @@ public void testStartMetricActorSystemRespectsThreadPriority() throws Exception
8380
final RpcService rpcService =
8481
MetricUtils.startRemoteMetricsRpcService(
8582
configuration, "localhost", RpcSystem.load());
86-
assertThat(rpcService, instanceOf(AkkaRpcService.class));
87-
88-
final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem();
8983

9084
try {
9185
final int threadPriority =
92-
actorSystem
93-
.settings()
94-
.config()
95-
.getInt("akka.actor.default-dispatcher.thread-priority");
96-
86+
rpcService.execute(() -> Thread.currentThread().getPriority()).get();
9787
assertThat(threadPriority, is(expectedThreadPriority));
9888
} finally {
9989
rpcService.stopService().get();

0 commit comments

Comments
 (0)