bytesandwich commented on a change in pull request #15898:
URL: https://github.com/apache/flink/pull/15898#discussion_r657430184
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java
##########
@@ -792,11 +801,140 @@ public void testHowToHandleFailureUnrecoverableFailure()
throws Exception {
assertThat(
scheduler
- .howToHandleFailure(new SuppressRestartsException(new
Exception("test")))
+ .howToHandleFailure(
+ null, new SuppressRestartsException(new
Exception("test")))
.canRestart(),
is(false));
}
+ @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 CountDownLatch latch = new CountDownLatch(1);
+ singleThreadMainThreadExecutor.execute(
+ () -> {
+ scheduler.handleGlobalFailure(expectedException);
+ latch.countDown();
+ });
+
+ latch.await();
+
+ Iterable<RootExceptionHistoryEntry> actualExceptionHistory =
+ scheduler.requestJob().getExceptionHistory();
+ final long end = System.currentTimeMillis();
+
+ assertThat(actualExceptionHistory,
IsIterableWithSize.iterableWithSize(1));
+
+ RootExceptionHistoryEntry failure =
actualExceptionHistory.iterator().next();
+
+ assertThat(
+
failure.getException().deserializeError(ClassLoader.getSystemClassLoader()),
+ Matchers.is(expectedException));
+ assertThat(failure.getTimestamp(), greaterThanOrEqualTo(start));
+ assertThat(failure.getTimestamp(), lessThanOrEqualTo(end));
+ assertThat(failure.getTaskManagerLocation(), Matchers.is(nullValue()));
+ assertThat(failure.getFailingTaskName(), Matchers.is(nullValue()));
+ }
+
+ @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 = 4;
+ 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 CountDownLatch latch = new CountDownLatch(1);
+ singleThreadMainThreadExecutor.execute(
+ () -> {
+ scheduler.updateTaskExecutionState(
+ new TaskExecutionStateTransition(
+ new TaskExecutionState(
+ attemptId, ExecutionState.FAILED,
expectedException)));
+ latch.countDown();
+ });
+
+ latch.await();
+
+ Iterable<RootExceptionHistoryEntry> actualExceptionHistory =
+ scheduler.requestJob().getExceptionHistory();
+ final long end = System.currentTimeMillis();
+
+ assertThat(actualExceptionHistory,
IsIterableWithSize.iterableWithSize(1));
+
+ RootExceptionHistoryEntry failure =
actualExceptionHistory.iterator().next();
+
+ assertThat(
+
failure.getException().deserializeError(ClassLoader.getSystemClassLoader()),
+ Matchers.is(expectedException));
+ assertThat(failure.getTimestamp(), greaterThanOrEqualTo(start));
+ assertThat(failure.getTimestamp(), lessThanOrEqualTo(end));
+ assertThat(failure.getTaskManagerLocation(), Matchers.is(nullValue()));
+ assertThat(failure.getFailingTaskName(), Matchers.is(nullValue()));
Review comment:
Do we need to add the code from `getFailureTimestamp` in
`DefaultSchedulerTest` to use that?
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]