metaswirl commented on a change in pull request #18689:
URL: https://github.com/apache/flink/pull/18689#discussion_r802824288



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java
##########
@@ -969,10 +988,213 @@ public void testHowToHandleFailureUnrecoverableFailure() 
throws Exception {
                 new AdaptiveSchedulerBuilder(createJobGraph(), 
mainThreadExecutor).build();
 
         assertThat(
-                scheduler
-                        .howToHandleFailure(new SuppressRestartsException(new 
Exception("test")))
-                        .canRestart(),
-                is(false));
+                        scheduler
+                                .howToHandleFailure(
+                                        new GlobalFailure(
+                                                new SuppressRestartsException(
+                                                        new 
Exception("test"))))
+                                .canRestart())
+                .isFalse();
+    }
+
+    @Test
+    public void testExceptionHistoryWithGlobalFailure() throws Exception {
+        final JobGraph jobGraph = createJobGraph();
+
+        final DefaultDeclarativeSlotPool declarativeSlotPool =
+                createDeclarativeSlotPool(jobGraph.getJobID());
+
+        final Configuration configuration = new Configuration();
+        configuration.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, 
Duration.ofMillis(1L));
+
+        final AdaptiveScheduler scheduler =
+                new AdaptiveSchedulerBuilder(jobGraph, 
singleThreadMainThreadExecutor)
+                        .setJobMasterConfiguration(configuration)
+                        .setDeclarativeSlotPool(declarativeSlotPool)
+                        .build();
+
+        final int numAvailableSlots = 1;
+        final SubmissionBufferingTaskManagerGateway taskManagerGateway =
+                new SubmissionBufferingTaskManagerGateway(numAvailableSlots);
+
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    scheduler.startScheduling();
+                    offerSlots(
+                            declarativeSlotPool,
+                            createSlotOffersForResourceRequirements(
+                                    ResourceCounter.withResource(
+                                            ResourceProfile.UNKNOWN, 
numAvailableSlots)),
+                            taskManagerGateway);
+                });
+        taskManagerGateway.waitForSubmissions(numAvailableSlots, 
Duration.ofSeconds(5));
+
+        final Exception expectedException = new Exception("Expected Global 
Exception");
+        final long start = System.currentTimeMillis();
+        final OneShotLatch latch = new OneShotLatch();
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    scheduler.handleGlobalFailure(expectedException);
+                    latch.trigger();
+                });
+
+        latch.await();
+
+        Iterable<RootExceptionHistoryEntry> actualExceptionHistory =
+                scheduler.requestJob().getExceptionHistory();
+        final long end = System.currentTimeMillis();
+
+        assertThat(actualExceptionHistory).hasSize(1);
+
+        RootExceptionHistoryEntry failure = 
actualExceptionHistory.iterator().next();
+
+        
assertThat(failure.getException().deserializeError(ClassLoader.getSystemClassLoader()))
+                .isEqualTo(expectedException);
+        assertThat(failure.getTimestamp()).isGreaterThanOrEqualTo(start);
+        assertThat(failure.getTimestamp()).isLessThanOrEqualTo(end);
+        assertThat(failure.getTaskManagerLocation()).isNull();
+        assertThat(failure.getFailingTaskName()).isNull();
+    }
+
+    @Test
+    public void testExceptionHistoryWithTaskFailure() throws Exception {
+        final JobGraph jobGraph = createJobGraph();
+
+        final DefaultDeclarativeSlotPool declarativeSlotPool =
+                createDeclarativeSlotPool(jobGraph.getJobID());
+
+        final Configuration configuration = new Configuration();
+        configuration.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, 
Duration.ofMillis(1L));
+
+        final AdaptiveScheduler scheduler =
+                new AdaptiveSchedulerBuilder(jobGraph, 
singleThreadMainThreadExecutor)
+                        .setJobMasterConfiguration(configuration)
+                        .setDeclarativeSlotPool(declarativeSlotPool)
+                        .build();
+
+        final int numAvailableSlots = 1;
+        final SubmissionBufferingTaskManagerGateway taskManagerGateway =
+                new SubmissionBufferingTaskManagerGateway(numAvailableSlots);
+
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    scheduler.startScheduling();
+                    offerSlots(
+                            declarativeSlotPool,
+                            createSlotOffersForResourceRequirements(
+                                    ResourceCounter.withResource(
+                                            ResourceProfile.UNKNOWN, 
numAvailableSlots)),
+                            taskManagerGateway);
+                });
+        taskManagerGateway.waitForSubmissions(numAvailableSlots, 
Duration.ofSeconds(5));
+
+        final Exception expectedException = new Exception("local failure");
+        Iterable<ArchivedExecutionVertex> executionVertices =
+                
scheduler.requestJob().getArchivedExecutionGraph().getAllExecutionVertices();
+
+        ExecutionAttemptID attemptId =
+                
executionVertices.iterator().next().getCurrentExecutionAttempt().getAttemptId();
+        final long start = System.currentTimeMillis();
+        final OneShotLatch latch = new OneShotLatch();
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    scheduler.updateTaskExecutionState(
+                            new TaskExecutionStateTransition(
+                                    new TaskExecutionState(
+                                            attemptId, ExecutionState.FAILED, 
expectedException)));
+                    latch.trigger();
+                });
+
+        latch.await();

Review comment:
       Could this lead to a race condition? Because the task executions are 
halted down asynchronously.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to