AHeise commented on a change in pull request #11948:
URL: https://github.com/apache/flink/pull/11948#discussion_r417896328
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -551,7 +551,7 @@ void onSenderBacklog(int backlog) throws IOException {
}
public void onBuffer(Buffer buffer, int sequenceNumber, int backlog)
throws IOException {
- boolean recycleBuffer = true;
+ boolean enqueued = false;
Review comment:
renaming and inversion is unnecessary. You could have just pulled up
setting `recycleBuffer = false`.
Note that this whole hotfix is rather academic. In case of some parse
exception, the whole TM is restarted anyways.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -291,7 +291,7 @@ public int unsynchronizedGetNumberOfQueuedBuffers() {
* returns null in all other cases.
*/
@Nullable
- protected CheckpointBarrier parseCheckpointBarrierOrNull(Buffer buffer)
throws IOException {
+ CheckpointBarrier parseCheckpointBarrierOrNull(Buffer buffer) throws
IOException {
Review comment:
Unnecessary change. `protected` was used to give all subclasses access
to this method. That there are currently all in the same package is rather
coincidental.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java
##########
@@ -108,18 +108,24 @@ private static CheckpointBarrierHandler
createCheckpointBarrierHandler(
AbstractInvokable toNotifyOnCheckpoint) {
switch (config.getCheckpointMode()) {
case EXACTLY_ONCE:
- if (config.isUnalignedCheckpointsEnabled()) {
- return new CheckpointBarrierUnaligner(
-
numberOfInputChannelsPerGate.toArray(),
- channelStateWriter,
+ return config.isUnalignedCheckpointsEnabled() ?
Review comment:
Again ternary is not making anything easier here.
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/AlternatingCheckpointBarrierHandlerTest.java
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointType;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.SAVEPOINT;
+import static
org.apache.flink.runtime.io.network.api.serialization.EventSerializer.toBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * {@link AlternatingCheckpointBarrierHandler} test.
+ */
+public class AlternatingCheckpointBarrierHandlerTest {
+
+ @Test
+ public void testCheckpointHandling() throws Exception {
+ testBarrierHandling(CHECKPOINT);
+ }
+
+ @Test
+ public void testSavepointHandling() throws Exception {
+ testBarrierHandling(SAVEPOINT);
+ }
+
+ @Test
+ public void testAlternation() throws Exception {
+ int numBarriers = 123;
+ int numChannels = 123;
+ TestInvokable target = new TestInvokable();
+ CheckpointedInputGate gate = buildGate(target, numChannels);
+ List<Long> barriers = new ArrayList<>();
+ for (long barrier = 0; barrier < numBarriers; barrier++) {
+ barriers.add(barrier);
+ CheckpointType type = barrier % 2 == 0 ? CHECKPOINT :
SAVEPOINT;
+ for (int channel = 0; channel < numChannels; channel++)
{
+ sendBarrier(barrier, type, (TestInputChannel)
gate.getChannel(channel), gate);
+ }
+ }
+ assertEquals(barriers, target.triggeredCheckpoints);
+ }
+
+ private void testBarrierHandling(CheckpointType checkpointType) throws
Exception {
+ final long barrierId = 123L;
+ TestInvokable target = new TestInvokable();
+ SingleInputGate gate = new
SingleInputGateBuilder().setNumberOfChannels(2).build();
+ TestInputChannel fast = new TestInputChannel(gate, 0);
+ TestInputChannel slow = new TestInputChannel(gate, 1);
+ gate.setInputChannels(fast, slow);
+ AlternatingCheckpointBarrierHandler barrierHandler =
barrierHandler(gate, target);
+ CheckpointedInputGate checkpointedGate = new
CheckpointedInputGate(gate, barrierHandler, 0 /* offset */);
+
+ sendBarrier(barrierId, checkpointType, fast, checkpointedGate);
+
+ assertEquals(checkpointType.isSavepoint(),
target.triggeredCheckpoints.isEmpty());
+ assertEquals(checkpointType.isSavepoint(),
barrierHandler.isBlocked(fast.getChannelIndex()));
+ assertFalse(barrierHandler.isBlocked(slow.getChannelIndex()));
+
+ sendBarrier(barrierId, checkpointType, slow, checkpointedGate);
+
+ assertEquals(singletonList(barrierId),
target.triggeredCheckpoints);
+ for (InputChannel channel : gate.getInputChannels().values()) {
+
assertFalse(barrierHandler.isBlocked(channel.getChannelIndex()));
+ assertEquals(
+ String.format("channel %d should be
resumed", channel.getChannelIndex()),
Review comment:
nit: indent
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/AlternatingCheckpointBarrierHandlerTest.java
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointType;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.SAVEPOINT;
+import static
org.apache.flink.runtime.io.network.api.serialization.EventSerializer.toBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * {@link AlternatingCheckpointBarrierHandler} test.
+ */
+public class AlternatingCheckpointBarrierHandlerTest {
Review comment:
Good tests! But I think you also need to test `BufferReceivedListener`
as the current version of AlternatingCheckpointBarrierHandler is probably not
allowing any UC on input side.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -575,12 +574,10 @@ public void onBuffer(Buffer buffer, int sequenceNumber,
int backlog) throws IOEx
receivedBuffers.add(buffer);
enqueued = true;
- if (listener != null && buffer.isBuffer() &&
receivedCheckpointId < lastRequestedCheckpointId) {
- notifyReceivedBuffer =
buffer.retainBuffer();
Review comment:
Even if you go optional, I wouldn't go ternary here. Ternary is meant
for some simple condition and one-liners.
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/AlternatingCheckpointBarrierHandlerTest.java
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointType;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.SAVEPOINT;
+import static
org.apache.flink.runtime.io.network.api.serialization.EventSerializer.toBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * {@link AlternatingCheckpointBarrierHandler} test.
+ */
+public class AlternatingCheckpointBarrierHandlerTest {
+
+ @Test
+ public void testCheckpointHandling() throws Exception {
+ testBarrierHandling(CHECKPOINT);
+ }
+
+ @Test
+ public void testSavepointHandling() throws Exception {
+ testBarrierHandling(SAVEPOINT);
+ }
+
+ @Test
+ public void testAlternation() throws Exception {
+ int numBarriers = 123;
+ int numChannels = 123;
+ TestInvokable target = new TestInvokable();
+ CheckpointedInputGate gate = buildGate(target, numChannels);
+ List<Long> barriers = new ArrayList<>();
+ for (long barrier = 0; barrier < numBarriers; barrier++) {
+ barriers.add(barrier);
+ CheckpointType type = barrier % 2 == 0 ? CHECKPOINT :
SAVEPOINT;
+ for (int channel = 0; channel < numChannels; channel++)
{
+ sendBarrier(barrier, type, (TestInputChannel)
gate.getChannel(channel), gate);
+ }
+ }
+ assertEquals(barriers, target.triggeredCheckpoints);
+ }
+
+ private void testBarrierHandling(CheckpointType checkpointType) throws
Exception {
+ final long barrierId = 123L;
+ TestInvokable target = new TestInvokable();
+ SingleInputGate gate = new
SingleInputGateBuilder().setNumberOfChannels(2).build();
+ TestInputChannel fast = new TestInputChannel(gate, 0);
+ TestInputChannel slow = new TestInputChannel(gate, 1);
+ gate.setInputChannels(fast, slow);
+ AlternatingCheckpointBarrierHandler barrierHandler =
barrierHandler(gate, target);
+ CheckpointedInputGate checkpointedGate = new
CheckpointedInputGate(gate, barrierHandler, 0 /* offset */);
+
+ sendBarrier(barrierId, checkpointType, fast, checkpointedGate);
+
+ assertEquals(checkpointType.isSavepoint(),
target.triggeredCheckpoints.isEmpty());
+ assertEquals(checkpointType.isSavepoint(),
barrierHandler.isBlocked(fast.getChannelIndex()));
+ assertFalse(barrierHandler.isBlocked(slow.getChannelIndex()));
+
+ sendBarrier(barrierId, checkpointType, slow, checkpointedGate);
+
+ assertEquals(singletonList(barrierId),
target.triggeredCheckpoints);
+ for (InputChannel channel : gate.getInputChannels().values()) {
+
assertFalse(barrierHandler.isBlocked(channel.getChannelIndex()));
+ assertEquals(
+ String.format("channel %d should be
resumed", channel.getChannelIndex()),
+ checkpointType.isSavepoint(),
+ ((TestInputChannel)
channel).wasResumed);
+ }
+ }
+
+ private void sendBarrier(long id, CheckpointType type, TestInputChannel
channel, CheckpointedInputGate gate) throws Exception {
+ channel.receive(barrier(id, type));
+ while (gate.pollNext().isPresent()) {
+ }
+ }
+
+ private static AlternatingCheckpointBarrierHandler
barrierHandler(SingleInputGate inputGate, AbstractInvokable target) {
+ String taskName = "test";
+ InputGate[] channelIndexToInputGate = new
InputGate[inputGate.getNumberOfInputChannels()];
+ Arrays.fill(channelIndexToInputGate, inputGate);
+ return new AlternatingCheckpointBarrierHandler(
+ new CheckpointBarrierAligner(taskName,
channelIndexToInputGate, singletonMap(inputGate, 0), target),
+ new CheckpointBarrierUnaligner(new
int[]{inputGate.getNumberOfInputChannels()}, ChannelStateWriter.NO_OP,
taskName, target),
+ target);
+ }
+
+ private Buffer barrier(long id, CheckpointType checkpointType) throws
IOException {
+ return toBuffer(new CheckpointBarrier(
+ id,
+ System.currentTimeMillis(),
+ new CheckpointOptions(checkpointType,
CheckpointStorageLocationReference.getDefault(), true, true)));
+ }
+
+ private static class TestInvokable extends AbstractInvokable {
+ private List<Long> triggeredCheckpoints = new ArrayList<>();
+
+ TestInvokable() {
+ super(new DummyEnvironment());
+ }
+
+ @Override
+ public void invoke() {
+ }
+
+ @Override
+ public <E extends Exception> void
executeInTaskThread(ThrowingRunnable<E> runnable, String descriptionFormat,
Object... descriptionArgs) throws E {
+ runnable.run();
+ }
+
+ @Override
+ public void triggerCheckpointOnBarrier(CheckpointMetaData
checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics
checkpointMetrics) {
+
triggeredCheckpoints.add(checkpointMetaData.getCheckpointId());
+ }
+ }
+
+ private static class TestInputChannel extends InputChannel {
+ private final Deque<Buffer> buffers = new LinkedList<>();
+ private boolean wasResumed;
+
+ TestInputChannel(SingleInputGate inputGate, int channelIndex) {
+ super(inputGate, channelIndex, new ResultPartitionID(),
0, 10, null, null);
+ }
+
+ public void receive(Buffer buffer) {
+ buffers.add(buffer);
+ notifyChannelNonEmpty();
+ }
+
+ @Override
+ public void requestSubpartition(int subpartitionIndex) {
+ }
+
+ @Override
+ protected Optional<BufferAndAvailability> getNextBuffer() {
+ return buffers.isEmpty() ? Optional.empty() :
Optional.of(new InputChannel.BufferAndAvailability(buffers.poll(),
buffers.isEmpty(), buffers.size()));
+ }
+
+ @Override
+ protected void sendTaskEvent(TaskEvent event) {
+ }
+
+ @Override
+ protected boolean isReleased() {
+ return false;
+ }
+
+ @Override
+ protected void releaseAllResources() {
+ }
+
+ @Override
+ public void resumeConsumption() {
+ wasResumed = true;
+ }
+ }
+
+ private static CheckpointedInputGate buildGate(TestInvokable target,
int numChannels) {
+ SingleInputGate gate = new
SingleInputGateBuilder().setNumberOfChannels(numChannels).build();
+ TestInputChannel[] channels = new TestInputChannel[numChannels];
+ for (int i = 0; i < numChannels; i++) {
+ channels[i] = new TestInputChannel(gate, i);
+ }
+ gate.setInputChannels(channels);
Review comment:
nit: could be a oneliner with (Int)Stream.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AlternatingCheckpointBarrierHandler.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+class AlternatingCheckpointBarrierHandler extends CheckpointBarrierHandler {
Review comment:
You most defintively need to delegate `getBufferReceivedListener`.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -560,8 +559,8 @@ public void onBuffer(Buffer buffer, int sequenceNumber, int
backlog) throws IOEx
}
final boolean wasEmpty;
- final CheckpointBarrier notifyReceivedBarrier;
- final Buffer notifyReceivedBuffer;
+ final Optional<CheckpointBarrier> notifyReceivedBarrier;
+ final Optional<Buffer> notifyReceivedBuffer;
Review comment:
While there is a good reason to use `Optional` for barrier as it's the
result of a function, I don't see the benefit of using it on buffer.
Did you do it for symmetry or was there a deeper reason?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
##########
@@ -221,16 +221,10 @@ private void prepareInflightDataSnapshot(long
checkpointId) throws IOException {
channelStateWriter.finishOutput(checkpointId);
}
- private void finishAndReportAsync(Map<OperatorID,
OperatorSnapshotFutures> snapshotFutures, CheckpointMetaData metadata,
CheckpointMetrics metrics) {
- final Future<?> channelWrittenFuture;
- if (unalignedCheckpointEnabled) {
- ChannelStateWriteResult writeResult =
channelStateWriter.getWriteResult(metadata.getCheckpointId());
- channelWrittenFuture = CompletableFuture.allOf(
- writeResult.getInputChannelStateHandles(),
-
writeResult.getResultSubpartitionStateHandles());
- } else {
- channelWrittenFuture =
FutureUtils.completedVoidFuture();
- }
+ private void finishAndReportAsync(Map<OperatorID,
OperatorSnapshotFutures> snapshotFutures, CheckpointMetaData metadata,
CheckpointMetrics metrics, CheckpointOptions options) {
+ Future<?> channelWrittenFuture = unalignedCheckpointEnabled &&
!options.getCheckpointType().isSavepoint() ?
Review comment:
Same on ternary.
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/AlternatingCheckpointBarrierHandlerTest.java
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.io;
+
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.CheckpointType;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.SAVEPOINT;
+import static
org.apache.flink.runtime.io.network.api.serialization.EventSerializer.toBuffer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * {@link AlternatingCheckpointBarrierHandler} test.
+ */
+public class AlternatingCheckpointBarrierHandlerTest {
+
+ @Test
+ public void testCheckpointHandling() throws Exception {
+ testBarrierHandling(CHECKPOINT);
+ }
+
+ @Test
+ public void testSavepointHandling() throws Exception {
+ testBarrierHandling(SAVEPOINT);
+ }
+
+ @Test
+ public void testAlternation() throws Exception {
Review comment:
I guess this test is super fast (<1s). If it takes longer, please reduce
`numChannels` a bit.
----------------------------------------------------------------
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]