fapaul commented on code in PR #21052:
URL: https://github.com/apache/flink/pull/21052#discussion_r996778934


##########
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/CheckpointCountingSource.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.util;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.function.Function;
+
+/**
+ * Each of the source operators outputs records in given number of 
checkpoints. Number of records
+ * per checkpoint is constant between checkpoints, and defined by user. When 
all records are
+ * emitted, the source waits for two more checkpoints until it finishes.
+ *
+ * <p>Main credits for this implementation should go to <b>Grzegorz
+ * Kolakowski/https://github.com/grzegorz8</b> who implemented the original 
version of this class
+ * for Delta-Flink connector.
+ */
+public class CheckpointCountingSource<OUT> extends 
RichParallelSourceFunction<OUT>
+        implements CheckpointListener, CheckpointedFunction {
+
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(CheckpointCountingSource.class);
+
+    private final int numberOfCheckpoints;
+    private final int recordsPerCheckpoint;
+    private final Function<Integer, OUT> elementProducer;
+
+    private ListState<Integer> nextValueState;
+    private int nextValue;
+    private volatile boolean isCanceled;
+    private volatile boolean waitingForCheckpoint;
+
+    public CheckpointCountingSource(
+            int recordsPerCheckpoint,
+            int numberOfCheckpoints,
+            Function<Integer, OUT> elementProducer) {
+        this.numberOfCheckpoints = numberOfCheckpoints;
+        this.recordsPerCheckpoint = recordsPerCheckpoint;
+        this.elementProducer = elementProducer;
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws 
Exception {
+        nextValueState =
+                context.getOperatorStateStore()
+                        .getListState(new ListStateDescriptor<>("nextValue", 
Integer.class));
+
+        if (nextValueState.get() != null && 
nextValueState.get().iterator().hasNext()) {
+            nextValue = nextValueState.get().iterator().next();
+        }
+        waitingForCheckpoint = false;
+    }
+
+    @Override
+    public void run(SourceContext<OUT> ctx) throws Exception {
+        LOGGER.info(
+                "Run subtask={}; attempt={}.",
+                getRuntimeContext().getIndexOfThisSubtask(),
+                getRuntimeContext().getAttemptNumber());
+
+        sendRecordsUntil(numberOfCheckpoints, ctx);
+        idleUntilNextCheckpoint(ctx);
+        LOGGER.info("Source task done; subtask={}.", 
getRuntimeContext().getIndexOfThisSubtask());
+    }
+
+    private void sendRecordsUntil(int targetCheckpoints, SourceContext<OUT> 
ctx)
+            throws InterruptedException {
+        while (!isCanceled && nextValue < targetCheckpoints * 
recordsPerCheckpoint) {
+            synchronized (ctx.getCheckpointLock()) {
+                emitRecordsBatch(recordsPerCheckpoint, ctx);
+                waitingForCheckpoint = true;
+            }
+            LOGGER.info(
+                    "Waiting for checkpoint to complete; subtask={}.",
+                    getRuntimeContext().getIndexOfThisSubtask());
+            while (waitingForCheckpoint) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    private void emitRecordsBatch(int batchSize, SourceContext<OUT> ctx) {
+        for (int i = 0; i < batchSize; ++i) {
+            OUT row = elementProducer.apply(nextValue++);
+            ctx.collect(row);
+        }
+
+        LOGGER.info(
+                "Emitted {} records (total {}); subtask={}.",
+                batchSize,
+                nextValue,
+                getRuntimeContext().getIndexOfThisSubtask());
+    }
+
+    private void idleUntilNextCheckpoint(SourceContext<OUT> ctx) throws 
InterruptedException {
+        // Idle until the next checkpoint completes to avoid any premature job 
termination and
+        // race conditions.
+        LOGGER.info(
+                "Waiting for an additional checkpoint to complete; 
subtask={}.",
+                getRuntimeContext().getIndexOfThisSubtask());
+        synchronized (ctx.getCheckpointLock()) {
+            waitingForCheckpoint = true;
+        }
+        while (waitingForCheckpoint) {
+            Thread.sleep(100);
+        }
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext context) throws 
Exception {
+        nextValueState.update(Collections.singletonList(nextValue));
+        LOGGER.info(
+                "state snapshot done; checkpointId={}; subtask={}.",
+                context.getCheckpointId(),
+                getRuntimeContext().getIndexOfThisSubtask());
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        waitingForCheckpoint = false;

Review Comment:
   Are you sure it is reliable to only use a `volatile` field here? I do not 
think that `notifyCheckpointComplete` uses the checkpoint lock so that might 
cause inconsistencies. I would use an `AtomicBoolean` here



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkITCase.java:
##########
@@ -113,6 +122,69 @@ public void init() {
         GLOBAL_COMMIT_QUEUE.clear();
     }
 
+    /**
+     * This test executes Sink operator with committer and global committer. 
The global committer
+     * throws exception on 3rd checkpoint (commitNumberToFailOn == 2). The 
local mini cluster is
+     * executing the recovery, we should expect no data loss. In this 
particular setup unique number
+     * of rows persisted by committer should be same as unique number of rows 
persisted by
+     * GlobalCommitter.
+     */
+    @Test
+    public void testGlobalCommitterNotMissingRecordsDuringRecovery() throws 
Exception {
+
+        final StreamExecutionEnvironment env = buildStreamEnv();
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
+        env.enableCheckpointing(5000L);
+        env.setParallelism(1);
+
+        final int commitNumberToFailOn = 2;

Review Comment:
   I think this test can be flaky. The problem is that there is no guarantee 
when `notifyCheckpointCompleted` is triggered on the committer, and 
globalcommitter just waiting for two checkpoints is not enough.
   
   Afaict in the current state, it can happen that the source finishes and no 
commit was triggered yet and only the final finish call triggers one commit.
   
   We need to make sure before the source finishes that the committer and 
globalcommitter have committed/done a failver.



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkITCase.java:
##########
@@ -113,6 +122,69 @@ public void init() {
         GLOBAL_COMMIT_QUEUE.clear();
     }
 
+    /**
+     * This test executes Sink operator with committer and global committer. 
The global committer
+     * throws exception on 3rd checkpoint (commitNumberToFailOn == 2). The 
local mini cluster is
+     * executing the recovery, we should expect no data loss. In this 
particular setup unique number
+     * of rows persisted by committer should be same as unique number of rows 
persisted by
+     * GlobalCommitter.
+     */
+    @Test
+    public void testGlobalCommitterNotMissingRecordsDuringRecovery() throws 
Exception {
+
+        final StreamExecutionEnvironment env = buildStreamEnv();
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
+        env.enableCheckpointing(5000L);

Review Comment:
   Nit: A faster checkpoint probably increases the test execution.



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkITCase.java:
##########
@@ -113,6 +122,69 @@ public void init() {
         GLOBAL_COMMIT_QUEUE.clear();
     }
 
+    /**
+     * This test executes Sink operator with committer and global committer. 
The global committer
+     * throws exception on 3rd checkpoint (commitNumberToFailOn == 2). The 
local mini cluster is
+     * executing the recovery, we should expect no data loss. In this 
particular setup unique number
+     * of rows persisted by committer should be same as unique number of rows 
persisted by
+     * GlobalCommitter.
+     */
+    @Test
+    public void testGlobalCommitterNotMissingRecordsDuringRecovery() throws 
Exception {
+
+        final StreamExecutionEnvironment env = buildStreamEnv();
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));

Review Comment:
   Why the delayed restart?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/TestSink.java:
##########
@@ -397,6 +400,54 @@ public void endOfInput() {
         }
     }
 
+    /**
+     * {@link GlobalCommitter} implementation that can throw on exception when 
processing
+     * checkpoint. Exception is thrown once.
+     */
+    public static class FailOnCommitGlobalCommitter extends 
DefaultGlobalCommitter {
+
+        private static final Logger LOG =
+                LoggerFactory.getLogger(FailOnCommitGlobalCommitter.class);
+
+        // local counter used to decide if exception should be thrown. This 
field has to be static,
+        // since we want to keep its value after Flink's local cluster 
recovery.
+        private static int globalCheckpointCounter = 0;
+
+        private final int commitNumberToFailOn;
+
+        /**
+         * Creates instance of FailOnCommitGlobalCommitter.
+         *
+         * @param checkpointNumberToFailOn number of checkpoints after which 
exception should be
+         *     thrown.
+         * @param queueSupplier queueSupplier.
+         */
+        public FailOnCommitGlobalCommitter(
+                int checkpointNumberToFailOn, Supplier<Queue<String>> 
queueSupplier) {
+            super(queueSupplier);
+            this.commitNumberToFailOn = checkpointNumberToFailOn;
+        }
+
+        @Override
+        public List<String> commit(List<String> committables) {
+            LOG.info(
+                    "Commit number "
+                            + globalCheckpointCounter
+                            + ", committables size "
+                            + committables.size());
+            try {
+                if (globalCheckpointCounter == commitNumberToFailOn) {
+                    throw new RuntimeException(
+                            "GlobalCommitter Desired Exception on checkpoint "
+                                    + globalCheckpointCounter);
+                }
+            } finally {
+                globalCheckpointCounter++;

Review Comment:
   I think the code logic is a bit confusing because the exception happens one 
commit too late i.e.
   
   globalCheckpointCounter, commitNumberToFailOn
   0 -> 2
   1 -> 2
   2 -> 3
   
   It takes 3 commits until the value is triggered. You can simply remove the 
`try` block and increment the global counter before the check.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollectorSerializerTest.java:
##########
@@ -134,55 +190,78 @@ void 
testAlignSubtaskCommittableManagerCheckpointWithCheckpointCommittableManage
                 .isEqualTo(committableManager.getCheckpointId());
     }
 
+    /**
+     * @param assertMessageHeading prefix used for assertion fail message.
+     * @param subtaskId subtaskId to get {@link SubtaskCommittableManager} 
from {@link
+     *     CheckpointCommittableManagerImpl}
+     * @param expectedNumberOfSubtasks expected number of subtasks for {@link 
CommittableSummary}
+     * @param committableCollector collector to get {@link 
CheckpointCommittableManager}s from.
+     * @param expectedCommittableSize expected number of {@link 
CheckpointCommittableManager}.
+     * @param expectedNumberOfPendingRequestsPerCommittable expected number of 
pending request per
+     *     {@link SubtaskCommittableManager}.
+     */
     private void assertCommittableCollector(
             String assertMessageHeading,
             int subtaskId,
-            int numberOfSubtasks,
-            CommittableCollector<Integer> committableCollector) {
+            int expectedNumberOfSubtasks,
+            CommittableCollector<Integer> committableCollector,
+            int expectedCommittableSize,
+            Object[] expectedNumberOfPendingRequestsPerCommittable) {
 
         assertAll(
                 assertMessageHeading,
                 () -> {
                     final Collection<CheckpointCommittableManagerImpl<Integer>>
                             checkpointCommittables =
                                     
committableCollector.getCheckpointCommittables();
-                    assertThat(checkpointCommittables).hasSize(2);
+                    
assertThat(checkpointCommittables).hasSize(expectedCommittableSize);
 
                     final Iterator<CheckpointCommittableManagerImpl<Integer>> 
committablesIterator =
                             checkpointCommittables.iterator();
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager1 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint1 =
-                            
checkpointCommittableManager1.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager1.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint1
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(1);
-                    
assertThat(subtaskCommittableManagerCheckpoint1.getSubtaskId())
-                            .isEqualTo(subtaskId);
-
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager2 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint2 =
-                            
checkpointCommittableManager2.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager2.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint2
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(2);
-                    
assertThat(subtaskCommittableManagerCheckpoint2.getSubtaskId())
-                            .isEqualTo(subtaskId);
+
+                    int i = 0;

Review Comment:
   Isn't `i` always `0`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollectorSerializerTest.java:
##########
@@ -134,55 +190,78 @@ void 
testAlignSubtaskCommittableManagerCheckpointWithCheckpointCommittableManage
                 .isEqualTo(committableManager.getCheckpointId());
     }
 
+    /**
+     * @param assertMessageHeading prefix used for assertion fail message.
+     * @param subtaskId subtaskId to get {@link SubtaskCommittableManager} 
from {@link
+     *     CheckpointCommittableManagerImpl}
+     * @param expectedNumberOfSubtasks expected number of subtasks for {@link 
CommittableSummary}
+     * @param committableCollector collector to get {@link 
CheckpointCommittableManager}s from.
+     * @param expectedCommittableSize expected number of {@link 
CheckpointCommittableManager}.
+     * @param expectedNumberOfPendingRequestsPerCommittable expected number of 
pending request per
+     *     {@link SubtaskCommittableManager}.
+     */
     private void assertCommittableCollector(
             String assertMessageHeading,
             int subtaskId,
-            int numberOfSubtasks,
-            CommittableCollector<Integer> committableCollector) {
+            int expectedNumberOfSubtasks,
+            CommittableCollector<Integer> committableCollector,
+            int expectedCommittableSize,
+            Object[] expectedNumberOfPendingRequestsPerCommittable) {

Review Comment:
   Why do you need the `Object[]` here? I would assume that in the test case, 
we know all types and do not need to use them. Please also avoid using arrays 
if possible.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to