|
20 | 20 | import org.apache.flink.api.common.JobID; |
21 | 21 | import org.apache.flink.api.java.tuple.Tuple2; |
22 | 22 | import org.apache.flink.api.java.tuple.Tuple6; |
| 23 | +import org.apache.flink.metrics.Gauge; |
23 | 24 | import org.apache.flink.runtime.clusterframework.types.AllocationID; |
24 | 25 | import org.apache.flink.runtime.clusterframework.types.ResourceID; |
25 | 26 | import org.apache.flink.runtime.clusterframework.types.ResourceProfile; |
26 | 27 | import org.apache.flink.runtime.clusterframework.types.SlotID; |
27 | 28 | import org.apache.flink.runtime.instance.InstanceID; |
28 | 29 | import org.apache.flink.runtime.messages.Acknowledge; |
| 30 | +import org.apache.flink.runtime.metrics.MetricNames; |
29 | 31 | import org.apache.flink.runtime.metrics.MetricRegistry; |
30 | 32 | import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup; |
31 | 33 | import org.apache.flink.runtime.metrics.util.TestingMetricRegistry; |
|
38 | 40 | import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; |
39 | 41 | import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; |
40 | 42 | import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; |
| 43 | +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; |
41 | 44 | import org.apache.flink.util.function.ThrowingConsumer; |
42 | 45 |
|
43 | 46 | import org.junit.jupiter.api.Test; |
|
50 | 53 | import java.util.Optional; |
51 | 54 | import java.util.concurrent.CompletableFuture; |
52 | 55 | import java.util.concurrent.atomic.AtomicInteger; |
| 56 | +import java.util.concurrent.atomic.AtomicReference; |
53 | 57 | import java.util.function.Consumer; |
54 | 58 |
|
55 | 59 | import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; |
@@ -1058,4 +1062,98 @@ void testClearResourceRequirementsWithPendingTaskManager() throws Exception { |
1058 | 1062 | } |
1059 | 1063 | }; |
1060 | 1064 | } |
| 1065 | + |
| 1066 | + @Test |
| 1067 | + void testMetricsUpdate() throws Exception { |
| 1068 | + final AtomicReference<Gauge<Long>> slotsAvailableGauge = new AtomicReference<>(); |
| 1069 | + final AtomicReference<Gauge<Long>> slotsTotalGauge = new AtomicReference<>(); |
| 1070 | + |
| 1071 | + final MetricRegistry metricRegistry = |
| 1072 | + TestingMetricRegistry.builder() |
| 1073 | + .setRegisterConsumer( |
| 1074 | + (metric, name, group) -> { |
| 1075 | + if (name.equals(MetricNames.TASK_SLOTS_AVAILABLE)) { |
| 1076 | + slotsAvailableGauge.set((Gauge<Long>) metric); |
| 1077 | + } else if (name.equals(MetricNames.TASK_SLOTS_TOTAL)) { |
| 1078 | + slotsTotalGauge.set((Gauge<Long>) metric); |
| 1079 | + } |
| 1080 | + }) |
| 1081 | + .build(); |
| 1082 | + |
| 1083 | + final Context context = new Context(); |
| 1084 | + context.setSlotManagerMetricGroup( |
| 1085 | + SlotManagerMetricGroup.create(metricRegistry, "localhost")); |
| 1086 | + final ManuallyTriggeredScheduledExecutor scheduledExecutor = |
| 1087 | + new ManuallyTriggeredScheduledExecutor(); |
| 1088 | + context.setScheduledExecutor(scheduledExecutor); |
| 1089 | + final TaskExecutorConnection executorConnection1 = createTaskExecutorConnection(); |
| 1090 | + final TaskExecutorConnection executorConnection2 = createTaskExecutorConnection(); |
| 1091 | + |
| 1092 | + context.runTest( |
| 1093 | + () -> { |
| 1094 | + assertThat(slotsAvailableGauge.get().getValue()).isEqualTo(0); |
| 1095 | + assertThat(slotsTotalGauge.get().getValue()).isEqualTo(0); |
| 1096 | + |
| 1097 | + final CompletableFuture<SlotManager.RegistrationResult> |
| 1098 | + registerTaskManagerFuture1 = new CompletableFuture<>(); |
| 1099 | + context.runInMainThreadAndWait( |
| 1100 | + () -> |
| 1101 | + registerTaskManagerFuture1.complete( |
| 1102 | + context.getSlotManager() |
| 1103 | + .registerTaskManager( |
| 1104 | + executorConnection1, |
| 1105 | + new SlotReport(), |
| 1106 | + DEFAULT_TOTAL_RESOURCE_PROFILE, |
| 1107 | + DEFAULT_SLOT_RESOURCE_PROFILE))); |
| 1108 | + assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture1)) |
| 1109 | + .isEqualTo(SlotManager.RegistrationResult.SUCCESS); |
| 1110 | + |
| 1111 | + final CompletableFuture<SlotManager.RegistrationResult> |
| 1112 | + registerTaskManagerFuture2 = new CompletableFuture<>(); |
| 1113 | + context.runInMainThreadAndWait( |
| 1114 | + () -> |
| 1115 | + registerTaskManagerFuture2.complete( |
| 1116 | + context.getSlotManager() |
| 1117 | + .registerTaskManager( |
| 1118 | + executorConnection2, |
| 1119 | + new SlotReport( |
| 1120 | + createAllocatedSlotStatus( |
| 1121 | + new JobID(), |
| 1122 | + new AllocationID(), |
| 1123 | + DEFAULT_SLOT_RESOURCE_PROFILE)), |
| 1124 | + DEFAULT_TOTAL_RESOURCE_PROFILE, |
| 1125 | + DEFAULT_SLOT_RESOURCE_PROFILE))); |
| 1126 | + assertThat(assertFutureCompleteAndReturn(registerTaskManagerFuture2)) |
| 1127 | + .isEqualTo(SlotManager.RegistrationResult.SUCCESS); |
| 1128 | + |
| 1129 | + // triggers the metric update task on the main thread |
| 1130 | + scheduledExecutor.triggerPeriodicScheduledTasks(); |
| 1131 | + context.runInMainThreadAndWait(() -> {}); |
| 1132 | + |
| 1133 | + assertThat(slotsTotalGauge.get().getValue()) |
| 1134 | + .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER); |
| 1135 | + assertThat(slotsAvailableGauge.get().getValue()) |
| 1136 | + .isEqualTo(2 * DEFAULT_NUM_SLOTS_PER_WORKER - 1); |
| 1137 | + |
| 1138 | + final CompletableFuture<Boolean> unRegisterTaskManagerFuture = |
| 1139 | + new CompletableFuture<>(); |
| 1140 | + context.runInMainThreadAndWait( |
| 1141 | + () -> |
| 1142 | + unRegisterTaskManagerFuture.complete( |
| 1143 | + context.getSlotManager() |
| 1144 | + .unregisterTaskManager( |
| 1145 | + executorConnection2.getInstanceID(), |
| 1146 | + TEST_EXCEPTION))); |
| 1147 | + assertThat(assertFutureCompleteAndReturn(unRegisterTaskManagerFuture)).isTrue(); |
| 1148 | + |
| 1149 | + // triggers the metric update task on the main thread |
| 1150 | + scheduledExecutor.triggerPeriodicScheduledTasks(); |
| 1151 | + context.runInMainThreadAndWait(() -> {}); |
| 1152 | + |
| 1153 | + assertThat(slotsTotalGauge.get().getValue()) |
| 1154 | + .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER); |
| 1155 | + assertThat(slotsAvailableGauge.get().getValue()) |
| 1156 | + .isEqualTo(DEFAULT_NUM_SLOTS_PER_WORKER); |
| 1157 | + }); |
| 1158 | + } |
1061 | 1159 | } |
0 commit comments