|
23 | 23 | import org.apache.flink.autoscaler.JobAutoScalerContext;
|
24 | 24 | import org.apache.flink.autoscaler.event.TestingEventCollector;
|
25 | 25 | import org.apache.flink.configuration.Configuration;
|
| 26 | +import org.apache.flink.util.concurrent.FutureUtils; |
26 | 27 |
|
27 | 28 | import org.junit.jupiter.api.Test;
|
28 | 29 | import org.junit.jupiter.params.ParameterizedTest;
|
|
33 | 34 | import java.util.Collections;
|
34 | 35 | import java.util.HashMap;
|
35 | 36 | import java.util.List;
|
| 37 | +import java.util.Map; |
36 | 38 | import java.util.Set;
|
37 | 39 | import java.util.concurrent.CompletableFuture;
|
| 40 | +import java.util.concurrent.ConcurrentHashMap; |
38 | 41 | import java.util.concurrent.CountDownLatch;
|
39 | 42 | import java.util.concurrent.atomic.AtomicLong;
|
40 | 43 |
|
@@ -227,6 +230,183 @@ public void cleanup(JobID jobID) {
|
227 | 230 | }
|
228 | 231 | }
|
229 | 232 |
|
| 233 | + @Test |
| 234 | + void testCleanupAfterStopped() throws Exception { |
| 235 | + var eventCollector = new TestingEventCollector<JobID, JobAutoScalerContext<JobID>>(); |
| 236 | + |
| 237 | + var job1 = createJobAutoScalerContext(); |
| 238 | + var job2 = createJobAutoScalerContext(); |
| 239 | + var scaleCounter = new ConcurrentHashMap<JobID, Integer>(); |
| 240 | + var cleanupCounter = new ConcurrentHashMap<JobID, Integer>(); |
| 241 | + |
| 242 | + var jobList = new ArrayList<JobAutoScalerContext<JobID>>(); |
| 243 | + |
| 244 | + try (var autoscalerExecutor = |
| 245 | + new StandaloneAutoscalerExecutor<>( |
| 246 | + new Configuration(), |
| 247 | + baseConf -> jobList, |
| 248 | + eventCollector, |
| 249 | + new JobAutoScaler<>() { |
| 250 | + @Override |
| 251 | + public void scale(JobAutoScalerContext<JobID> context) { |
| 252 | + scaleCounter.put( |
| 253 | + context.getJobKey(), |
| 254 | + scaleCounter.getOrDefault(context.getJobKey(), 0) + 1); |
| 255 | + } |
| 256 | + |
| 257 | + @Override |
| 258 | + public void cleanup(JobID jobID) { |
| 259 | + cleanupCounter.put( |
| 260 | + jobID, cleanupCounter.getOrDefault(jobID, 0) + 1); |
| 261 | + } |
| 262 | + })) { |
| 263 | + |
| 264 | + // Test for empty job list. |
| 265 | + var scaledFutures = autoscalerExecutor.scaling(); |
| 266 | + assertThat(scaledFutures).isEmpty(); |
| 267 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 268 | + assertThat(scaleCounter).isEmpty(); |
| 269 | + assertThat(cleanupCounter).isEmpty(); |
| 270 | + |
| 271 | + // Test for 2 jobs twice. |
| 272 | + jobList.add(job1); |
| 273 | + jobList.add(job2); |
| 274 | + |
| 275 | + scaledFutures = autoscalerExecutor.scaling(); |
| 276 | + assertThat(scaledFutures).hasSize(2); |
| 277 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 278 | + assertThat(scaleCounter) |
| 279 | + .containsExactlyInAnyOrderEntriesOf( |
| 280 | + Map.of(job1.getJobKey(), 1, job2.getJobKey(), 1)); |
| 281 | + assertThat(cleanupCounter).isEmpty(); |
| 282 | + |
| 283 | + scaledFutures = autoscalerExecutor.scaling(); |
| 284 | + assertThat(scaledFutures).hasSize(2); |
| 285 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 286 | + assertThat(scaleCounter) |
| 287 | + .containsExactlyInAnyOrderEntriesOf( |
| 288 | + Map.of(job1.getJobKey(), 2, job2.getJobKey(), 2)); |
| 289 | + assertThat(cleanupCounter).isEmpty(); |
| 290 | + |
| 291 | + // Test for 1 job twice. |
| 292 | + jobList.clear(); |
| 293 | + jobList.add(job2); |
| 294 | + |
| 295 | + scaledFutures = autoscalerExecutor.scaling(); |
| 296 | + assertThat(scaledFutures).hasSize(1); |
| 297 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 298 | + assertThat(scaleCounter) |
| 299 | + .containsExactlyInAnyOrderEntriesOf( |
| 300 | + Map.of(job1.getJobKey(), 2, job2.getJobKey(), 3)); |
| 301 | + assertThat(cleanupCounter) |
| 302 | + .containsExactlyInAnyOrderEntriesOf(Map.of(job1.getJobKey(), 1)); |
| 303 | + |
| 304 | + scaledFutures = autoscalerExecutor.scaling(); |
| 305 | + assertThat(scaledFutures).hasSize(1); |
| 306 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 307 | + assertThat(scaleCounter) |
| 308 | + .containsExactlyInAnyOrderEntriesOf( |
| 309 | + Map.of(job1.getJobKey(), 2, job2.getJobKey(), 4)); |
| 310 | + // Only cleanup once. |
| 311 | + assertThat(cleanupCounter) |
| 312 | + .containsExactlyInAnyOrderEntriesOf(Map.of(job1.getJobKey(), 1)); |
| 313 | + } |
| 314 | + } |
| 315 | + |
| 316 | + @Test |
| 317 | + void testCleanupForStoppedJobAfterScaling() throws Exception { |
| 318 | + var eventCollector = new TestingEventCollector<JobID, JobAutoScalerContext<JobID>>(); |
| 319 | + |
| 320 | + var job1 = createJobAutoScalerContext(); |
| 321 | + var job2 = createJobAutoScalerContext(); |
| 322 | + var scaleCounter = new ConcurrentHashMap<JobID, Integer>(); |
| 323 | + var cleanupCounter = new ConcurrentHashMap<JobID, Integer>(); |
| 324 | + |
| 325 | + var job1StartWaitFuture = new CompletableFuture<>(); |
| 326 | + var job1WaitFuture = new CompletableFuture<>(); |
| 327 | + |
| 328 | + var jobList = new ArrayList<JobAutoScalerContext<JobID>>(); |
| 329 | + |
| 330 | + try (var autoscalerExecutor = |
| 331 | + new StandaloneAutoscalerExecutor<>( |
| 332 | + new Configuration(), |
| 333 | + baseConf -> jobList, |
| 334 | + eventCollector, |
| 335 | + new JobAutoScaler<>() { |
| 336 | + @Override |
| 337 | + public void scale(JobAutoScalerContext<JobID> context) |
| 338 | + throws Exception { |
| 339 | + scaleCounter.put( |
| 340 | + context.getJobKey(), |
| 341 | + scaleCounter.getOrDefault(context.getJobKey(), 0) + 1); |
| 342 | + if (context == job1) { |
| 343 | + job1StartWaitFuture.complete(null); |
| 344 | + job1WaitFuture.get(); |
| 345 | + } |
| 346 | + } |
| 347 | + |
| 348 | + @Override |
| 349 | + public void cleanup(JobID jobID) { |
| 350 | + cleanupCounter.put( |
| 351 | + jobID, cleanupCounter.getOrDefault(jobID, 0) + 1); |
| 352 | + } |
| 353 | + })) { |
| 354 | + |
| 355 | + // Test for job1 and job2 |
| 356 | + jobList.add(job1); |
| 357 | + jobList.add(job2); |
| 358 | + |
| 359 | + var scaledFutures = autoscalerExecutor.scaling(); |
| 360 | + var job1ScaledFuture = scaledFutures.get(0); |
| 361 | + |
| 362 | + assertThat(scaledFutures).hasSize(2); |
| 363 | + // wait for job2 scaling is finished. |
| 364 | + scaledFutures.get(1).get(); |
| 365 | + // wait for job1 starts wait. |
| 366 | + job1StartWaitFuture.get(); |
| 367 | + assertThat(scaleCounter) |
| 368 | + .containsExactlyInAnyOrderEntriesOf( |
| 369 | + Map.of(job1.getJobKey(), 1, job2.getJobKey(), 1)); |
| 370 | + assertThat(cleanupCounter).isEmpty(); |
| 371 | + |
| 372 | + scalingOnlyHappenForJob2( |
| 373 | + job1, job2, scaleCounter, cleanupCounter, autoscalerExecutor, 2); |
| 374 | + |
| 375 | + // Test for job2 twice, and job1 should be clean up after scaling. |
| 376 | + jobList.clear(); |
| 377 | + jobList.add(job2); |
| 378 | + |
| 379 | + scalingOnlyHappenForJob2( |
| 380 | + job1, job2, scaleCounter, cleanupCounter, autoscalerExecutor, 3); |
| 381 | + scalingOnlyHappenForJob2( |
| 382 | + job1, job2, scaleCounter, cleanupCounter, autoscalerExecutor, 4); |
| 383 | + |
| 384 | + // Wait for job1 scaling to complete. |
| 385 | + job1WaitFuture.complete(null); |
| 386 | + job1ScaledFuture.get(); |
| 387 | + assertThat(cleanupCounter) |
| 388 | + .containsExactlyInAnyOrderEntriesOf(Map.of(job1.getJobKey(), 1)); |
| 389 | + } |
| 390 | + } |
| 391 | + |
| 392 | + private void scalingOnlyHappenForJob2( |
| 393 | + JobAutoScalerContext<JobID> job1, |
| 394 | + JobAutoScalerContext<JobID> job2, |
| 395 | + ConcurrentHashMap<JobID, Integer> scaleCounter, |
| 396 | + ConcurrentHashMap<JobID, Integer> cleanupCounter, |
| 397 | + StandaloneAutoscalerExecutor<JobID, JobAutoScalerContext<JobID>> autoscalerExecutor, |
| 398 | + int expectedJob2ScaleCounter) |
| 399 | + throws Exception { |
| 400 | + var scaledFutures = autoscalerExecutor.scaling(); |
| 401 | + assertThat(scaledFutures).hasSize(1); |
| 402 | + FutureUtils.waitForAll(scaledFutures).get(); |
| 403 | + assertThat(scaleCounter) |
| 404 | + .containsExactlyInAnyOrderEntriesOf( |
| 405 | + Map.of(job1.getJobKey(), 1, job2.getJobKey(), expectedJob2ScaleCounter)); |
| 406 | + // job1 is still scaling, it cannot be cleaned up. |
| 407 | + assertThat(cleanupCounter).isEmpty(); |
| 408 | + } |
| 409 | + |
230 | 410 | private static JobAutoScalerContext<JobID> createJobAutoScalerContext() {
|
231 | 411 | var jobID = new JobID();
|
232 | 412 | return new JobAutoScalerContext<>(
|
|
0 commit comments