pnowojski commented on a change in pull request #15313: URL: https://github.com/apache/flink/pull/15313#discussion_r601458195
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/AlignedCheckpointState.java ########## @@ -0,0 +1,82 @@ +/* + * 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.checkpointing; + +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.partition.consumer.CheckpointableInput; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** A state pojo for {@link AbstractAlignedBarrierHandlerAction}. */ +final class AlignedCheckpointState { Review comment: It was a bit confusing for me that there is no version of this class for unaligned checkpoint? The unaligned would have just one field less? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/checkpointing/BarrierHandlerAction.java ########## @@ -0,0 +1,66 @@ +/* + * 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.checkpointing; + +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; + +import java.io.IOException; + +/** + * Represents a state in a state machine of processing a checkpoint. There are 4 base states: + * + * <ul> + * <li>Waiting for an aligned barrier + * <li>Collecting aligned barriers + * <li>Waiting for an unaligned barrier + * <li>Collecting unaligned barriers + * </ul> + * + * <p>Additionally depending on the configuration we can switch between aligned and unaligned + * actions. + */ +interface BarrierHandlerAction { + BarrierHandlerAction alignmentTimeout(Context context, CheckpointBarrier checkpointBarrier) + throws IOException, CheckpointException; + + BarrierHandlerAction announcementReceived( + Context context, InputChannelInfo channelInfo, int sequenceNumber) throws IOException; + + BarrierHandlerAction barrierReceived( + Context context, InputChannelInfo channelInfo, CheckpointBarrier checkpointBarrier) + throws IOException, CheckpointException; + + BarrierHandlerAction abort(long cancelledId) throws IOException; + + /** + * An entry point for communication between {@link BarrierHandlerAction} and {@link + * SingleCheckpointBarrierHandler}. + */ + interface Context { + boolean allBarriersReceived(); + + void triggerGlobalCheckpoint(CheckpointBarrier checkpointBarrier) throws IOException; + + void triggerTaskCheckpoint(CheckpointBarrier checkpointBarrier) throws CheckpointException; Review comment: What is a difference between those two? edit: I think the names might be strange? The "global" one triggers checkpoint and performs actual checkpoint. The "task" one, is doing just some initialisation work. So maybe we should name them just that? `triggerCheckpoint` and `initCheckpoint`? ########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/AlignedCheckpointsTest.java ########## @@ -157,35 +150,6 @@ public void ensureEmpty() throws Exception { // Tests // ------------------------------------------------------------------------ - public void testGetChannelsWithUnprocessedBarriers() throws IOException { - mockInputGate = new MockInputGate(4, Collections.emptyList()); - AlignedController alignedController = new AlignedController(mockInputGate); - BufferOrEvent barrier0 = createBarrier(1, 0); - BufferOrEvent barrier1 = createBarrier(1, 1); - BufferOrEvent barrier3 = createBarrier(1, 3); - alignedController.barrierAnnouncement( - barrier0.getChannelInfo(), (CheckpointBarrier) barrier0.getEvent(), 0); - alignedController.barrierReceived( - barrier0.getChannelInfo(), (CheckpointBarrier) barrier0.getEvent()); - alignedController.barrierAnnouncement( - barrier1.getChannelInfo(), (CheckpointBarrier) barrier1.getEvent(), 1); - alignedController.barrierAnnouncement( - barrier3.getChannelInfo(), (CheckpointBarrier) barrier3.getEvent(), 42); - - Collection<InputChannelInfo> blockedChannels = alignedController.getBlockedChannels(); - Map<InputChannelInfo, Integer> announcedChannels = - alignedController.getSequenceNumberInAnnouncedChannels(); - - // blockedChannels and announcedChannels should be copies and shouldn't be cleared by the - // resumeConsumption - alignedController.resumeConsumption(); - - assertThat(blockedChannels, contains(barrier0.getChannelInfo())); - assertThat(announcedChannels, IsMapContaining.hasEntry(barrier1.getChannelInfo(), 1)); - assertThat(announcedChannels, IsMapContaining.hasEntry(barrier3.getChannelInfo(), 42)); - assertThat(announcedChannels.size(), equalTo(2)); - } - Review comment: What has happened with this test? Have you replaced it with something else? ########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/AlternatingCheckpointsTest.java ########## @@ -376,119 +586,26 @@ public void testTimeoutAlignmentOnUnalignedCheckpoint() throws Exception { assertBarrier(gate); assertEquals( - channelStateWriter.getAddedInput().get(getChannel(gate, 1).getChannelInfo()).size(), - 2); + 2, + channelStateWriter + .getAddedInput() + .get(getChannel(gate, 1).getChannelInfo()) + .size()); assertEquals(1, target.getTriggeredCheckpointCounter()); } private RemoteInputChannel getChannel(CheckpointedInputGate gate, int channelIndex) { return (RemoteInputChannel) gate.getChannel(channelIndex); } - @Test - public void testTimeoutAlignmentConsistencyOnPreProcessBarrier() throws Exception { - testTimeoutAlignmentConsistency(true, false, false); - } - - @Test - public void testTimeoutAlignmentConsistencyOnProcessBarrier() throws Exception { - testTimeoutAlignmentConsistency(false, true, false); - } - - @Test - public void testTimeoutAlignmentConsistencyOnPostProcessBarrier() throws Exception { - testTimeoutAlignmentConsistency(false, false, true); - } - - public void testTimeoutAlignmentConsistency( - boolean sleepBeforePreProcess, - boolean sleepBeforeProcess, - boolean sleepBeforePostProcess) - throws Exception { - SingleInputGate gate = new SingleInputGateBuilder().setNumberOfChannels(1).build(); - TestInputChannel channel0 = new TestInputChannel(gate, 0, false, true); - gate.setInputChannels(channel0); - - RecordingChannelStateWriter channelStateWriter = new RecordingChannelStateWriter(); - AlternatingController controller = - new AlternatingController( - new AlignedController(gate), - new UnalignedController( - new TestSubtaskCheckpointCoordinator(channelStateWriter), gate), - clock); - - long alignmentTimeout = 10; - CheckpointBarrier barrier = - new CheckpointBarrier( - 1, clock.relativeTimeMillis(), alignedNoTimeout(CHECKPOINT, getDefault())); - - InputChannelInfo channelInfo = channel0.getChannelInfo(); - - controller.preProcessFirstBarrierOrAnnouncement(barrier); - controller.barrierAnnouncement(channelInfo, barrier, 1); - - if (sleepBeforePreProcess) { - clock.advanceTime(alignmentTimeout * 2, TimeUnit.MILLISECONDS); - } - Optional<CheckpointBarrier> preProcessTrigger = - controller.preProcessFirstBarrier(channelInfo, barrier); - if (sleepBeforeProcess) { - clock.advanceTime(alignmentTimeout * 2, TimeUnit.MILLISECONDS); - } - Optional<CheckpointBarrier> processTrigger = - controller.barrierReceived(channelInfo, barrier); - if (sleepBeforePostProcess) { - clock.advanceTime(alignmentTimeout * 2, TimeUnit.MILLISECONDS); - } - Optional<CheckpointBarrier> postProcessTrigger = - controller.postProcessLastBarrier(channelInfo, barrier); - - int triggeredCount = 0; - boolean unalignedCheckpoint = false; - if (preProcessTrigger.isPresent()) { - triggeredCount++; - unalignedCheckpoint = - preProcessTrigger.get().getCheckpointOptions().isUnalignedCheckpoint(); - assertTrue(unalignedCheckpoint); - } - if (processTrigger.isPresent()) { - triggeredCount++; - unalignedCheckpoint = - processTrigger.get().getCheckpointOptions().isUnalignedCheckpoint(); - assertTrue(unalignedCheckpoint); - } - if (postProcessTrigger.isPresent()) { - triggeredCount++; - unalignedCheckpoint = - postProcessTrigger.get().getCheckpointOptions().isUnalignedCheckpoint(); - } - - assertEquals( - String.format( - "Checkpoint should be triggered exactly once, but [%s, %s, %s] was found instead", - preProcessTrigger.isPresent(), - processTrigger.isPresent(), - postProcessTrigger.isPresent()), - 1, - triggeredCount); - - if (unalignedCheckpoint) { - // check that we can add output data if we are in unaligned checkpoint mode. In other - // words - // if the state writer has been initialised correctly. - assertEquals(barrier.getId(), channelStateWriter.getLastStartedCheckpointId()); - } - } Review comment: Was this test re-written? -- 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]
