zhijiangW commented on a change in pull request #12664:
URL: https://github.com/apache/flink/pull/12664#discussion_r440597301
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java
##########
@@ -218,6 +230,67 @@ public void testNotifyCheckpointAbortedBeforeAsyncPhase()
throws Exception {
assertEquals(0,
subtaskCheckpointCoordinator.getAsyncCheckpointRunnableSize());
}
+ @Test
+ public void
testDownstreamReceiveCancelCheckpointMarkerOnUpstreamAbortedInSyncPhase()
throws Exception {
+ final OneInputStreamTaskTestHarness<String, String> testHarness
=
+ new OneInputStreamTaskTestHarness<>(
+ OneInputStreamTask::new,
+ 1, 1,
+ BasicTypeInfo.STRING_TYPE_INFO,
+ BasicTypeInfo.STRING_TYPE_INFO);
+
+ testHarness.setupOutputForSingletonOperatorChain();
+ StreamConfig streamConfig = testHarness.getStreamConfig();
+ streamConfig.setStreamOperator(new MapOperator());
+
+ testHarness.invoke();
+ testHarness.waitForTaskRunning();
+
+ TestTaskStateManager stateManager = new TestTaskStateManager();
+ MockEnvironment mockEnvironment =
MockEnvironment.builder().setTaskStateManager(stateManager).build();
+ SubtaskCheckpointCoordinatorImpl subtaskCheckpointCoordinator =
(SubtaskCheckpointCoordinatorImpl) new MockSubtaskCheckpointCoordinatorBuilder()
+ .setEnvironment(mockEnvironment)
+ .setUnalignedCheckpointEnabled(true)
+ .build();
+
+ final TestPooledBufferProvider bufferProvider = new
TestPooledBufferProvider(Integer.MAX_VALUE, 4096);
+ ArrayList<Object> recordOrEvents = new ArrayList<>();
+ StreamElementSerializer<String> stringStreamElementSerializer =
new StreamElementSerializer<>(StringSerializer.INSTANCE);
+ RecordOrEventCollectingResultPartitionWriter<StreamElement>
resultPartitionWriter = new
RecordOrEventCollectingResultPartitionWriter<>(recordOrEvents, bufferProvider,
stringStreamElementSerializer);
+
mockEnvironment.addOutputs(Collections.singletonList(resultPartitionWriter));
+
+ OneInputStreamTask<String, String> task = testHarness.getTask();
+ final OperatorChain<String, OneInputStreamOperator<String,
String>> operatorChain = new OperatorChain<>(task,
StreamTask.createRecordWriterDelegate(streamConfig, mockEnvironment));
+ long checkpointId = 42L;
+ // notify checkpoint aborted before execution.
+
subtaskCheckpointCoordinator.notifyCheckpointAborted(checkpointId,
operatorChain, () -> true);
+
subtaskCheckpointCoordinator.getChannelStateWriter().start(checkpointId,
CheckpointOptions.forCheckpointWithDefaultLocation());
+ subtaskCheckpointCoordinator.checkpointState(
+ new CheckpointMetaData(checkpointId,
System.currentTimeMillis()),
+ CheckpointOptions.forCheckpointWithDefaultLocation(),
+ new CheckpointMetrics(),
+ operatorChain,
+ () -> true);
+
+ assertEquals(1, recordOrEvents.size());
+ Object recordOrEvent = recordOrEvents.get(0);
+ // ensure CancelCheckpointMarker is broadcast downstream.
+ assertTrue(recordOrEvent instanceof CancelCheckpointMarker);
+ assertEquals(checkpointId, ((CancelCheckpointMarker)
recordOrEvent).getCheckpointId());
+ }
Review comment:
We should make sure the internal task thread inside
`StreamTaskTestHarness` exit at last to avoid remaining thread after test
finishes?
----------------------------------------------------------------
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]