pnowojski commented on a change in pull request #12470:
URL: https://github.com/apache/flink/pull/12470#discussion_r435404303
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -919,9 +920,33 @@ public final ExecutorService
getAsyncOperationsThreadPool() {
@Override
public Future<Void> notifyCheckpointCompleteAsync(long checkpointId) {
- return
mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).submit(
- () -> notifyCheckpointComplete(checkpointId),
- "checkpoint %d complete", checkpointId);
+ return notifyCheckpointOperation(
+ () -> notifyCheckpointComplete(checkpointId),
+ String.format("checkpoint %d complete", checkpointId));
+ }
+
+ @Override
+ public Future<Void> notifyCheckpointAbortAsync(long checkpointId) {
+ return notifyCheckpointOperation(
+ () ->
subtaskCheckpointCoordinator.notifyCheckpointAborted(checkpointId,
operatorChain, this::isRunning),
+ String.format("checkpoint %d aborted", checkpointId));
+ }
+
+ private Future<Void> notifyCheckpointOperation(RunnableWithException
runnable, String description) {
Review comment:
Could you re-use this method in
`org.apache.flink.streaming.runtime.tasks.StreamTask#triggerCheckpointAsync`?
However I see there is a difference in the used mailbox executor (different
priority), so if this would cause too much extra complexity I'm fine with
leaving it as it is.
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
##########
@@ -969,6 +970,34 @@ public void testNotifyCheckpointOnClosedOperator() throws
Throwable {
assertEquals(true, operator.closed.get());
}
+ @Test
+ public void testFailToConfirmCheckpointCompleted() throws Exception {
+ testFailToConfirmCheckpointMessage(streamTask ->
streamTask.notifyCheckpointCompleteAsync(1L));
+ }
+
+ @Test
+ public void testFailToConfirmCheckpointAborted() throws Exception {
+ testFailToConfirmCheckpointMessage(streamTask ->
streamTask.notifyCheckpointAbortAsync(1L));
+ }
+
+ private void testFailToConfirmCheckpointMessage(Consumer<StreamTask<?,
?>> consumer) throws Exception {
+ FailOnNotifyCheckpointOperator<Integer> operator = new
FailOnNotifyCheckpointOperator<>();
+ MultipleInputStreamTaskTestHarnessBuilder<Integer> builder =
+ new
MultipleInputStreamTaskTestHarnessBuilder<>(OneInputStreamTask::new,
BasicTypeInfo.INT_TYPE_INFO)
+ .addInput(BasicTypeInfo.INT_TYPE_INFO);
+ StreamTaskMailboxTestHarness<Integer> harness = builder
+ .setupOutputForSingletonOperatorChain(operator)
+ .build();
+
+ try {
+ consumer.accept(harness.streamTask);
+ harness.streamTask.runMailboxStep();
+ fail();
+ } catch (ExpectedTestException expected) {
+ // expected exception
Review comment:
I presume that this test was failing on master branch (without your fix)?
----------------------------------------------------------------
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]