Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -235,9 +235,11 @@ public void checkpointState(
return;
}

// Step (0): Record the last triggered checkpointId.
// Step (0): Record the last triggered checkpointId and abort the sync phase of checkpoint if necessary.
lastCheckpointId = metadata.getCheckpointId();
if (checkAndClearAbortedStatus(metadata.getCheckpointId())) {
// broadcast cancel checkpoint marker to avoid downstream back-pressure due to checkpoint barrier align.
operatorChain.broadcastEvent(new CancelCheckpointMarker(metadata.getCheckpointId()));
LOG.info("Checkpoint {} has been notified as aborted, would not trigger any checkpoint.", metadata.getCheckpointId());
return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@

package org.apache.flink.streaming.runtime.tasks;

import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.metrics.MetricGroup;
Expand All @@ -28,7 +31,11 @@
import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
import org.apache.flink.runtime.io.network.api.writer.NonRecordWriter;
import org.apache.flink.runtime.io.network.api.writer.RecordOrEventCollectingResultPartitionWriter;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
Expand All @@ -39,12 +46,15 @@
import org.apache.flink.runtime.state.SnapshotResult;
import org.apache.flink.runtime.state.TestCheckpointStorageWorkerView;
import org.apache.flink.runtime.state.TestTaskStateManager;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
import org.apache.flink.streaming.api.operators.StreamMap;
import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask;
import org.apache.flink.streaming.util.MockStreamTaskBuilder;
Expand All @@ -53,6 +63,8 @@
import org.junit.Test;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
Expand Down Expand Up @@ -218,6 +230,69 @@ public void testNotifyCheckpointAbortedBeforeAsyncPhase() throws Exception {
assertEquals(0, subtaskCheckpointCoordinator.getAsyncCheckpointRunnableSize());
}

@Test
public void testBroadcastCancelCheckpointMarkerOnAbortingFromCoordinator() throws Exception {
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();

MockEnvironment mockEnvironment = MockEnvironment.builder().build();
SubtaskCheckpointCoordinator subtaskCheckpointCoordinator = new MockSubtaskCheckpointCoordinatorBuilder()
.setEnvironment(mockEnvironment)
.build();

TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(1, 4096);
ArrayList<Object> recordOrEvents = new ArrayList<>();
StreamElementSerializer<String> stringStreamElementSerializer = new StreamElementSerializer<>(StringSerializer.INSTANCE);
ResultPartitionWriter resultPartitionWriter = new RecordOrEventCollectingResultPartitionWriter<>(
recordOrEvents, bufferProvider, stringStreamElementSerializer);
mockEnvironment.addOutputs(Collections.singletonList(resultPartitionWriter));

OneInputStreamTask<String, String> task = testHarness.getTask();
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.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());
testHarness.endInput();
testHarness.waitForTaskCompletion();
}

private static class MapOperator extends StreamMap<String, String> {
private static final long serialVersionUID = 1L;

public MapOperator() {
super((MapFunction<String, String>) value -> value);
}

@Override
public void notifyCheckpointAborted(long checkpointId) throws Exception {
}
}

@Test
public void testNotifyCheckpointAbortedDuringAsyncPhase() throws Exception {
MockEnvironment mockEnvironment = MockEnvironment.builder().build();
Expand Down