rkhachatryan commented on a change in pull request #18224:
URL: https://github.com/apache/flink/pull/18224#discussion_r807317423



##########
File path: 
flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.test.checkpointing;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.StateChangelogOptions;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.function.FutureTaskWithException;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.Duration;
+import java.util.concurrent.Future;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This verifies that checkpointing works correctly for Changelog state 
backend with materialized
+ * state / non-materialized state.
+ */
+public class ChangelogPeriodicMaterializationITCase
+        extends ChangelogPeriodicMaterializationTestBase {
+
+    private static final AtomicBoolean triggerDelegatedSnapshot = new 
AtomicBoolean();
+
+    public ChangelogPeriodicMaterializationITCase(AbstractStateBackend 
delegatedStateBackend) {
+        super(delegatedStateBackend);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        triggerDelegatedSnapshot.set(false);
+    }
+
+    /** Recovery from checkpoint only containing non-materialized state. */
+    @Test
+    public void testNonMaterialization() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(10)
+                .enableChangelogStateBackend(true)
+                .getCheckpointConfig()
+                .setCheckpointStorage(TEMPORARY_FOLDER.newFolder().toURI());
+        env.setStateBackend(new 
DelegatedStateBackendWrapper(delegatedStateBackend, t -> t))
+                .setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+        env.configure(
+                new Configuration()
+                        .set(
+                                
StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL,
+                                Duration.ofMinutes(3)));
+        waitAndAssert(
+                env,
+                buildStreamGraph(
+                        env,
+                        new ControlledSource() {
+                            @Override
+                            protected void beforeElement() throws Exception {
+                                if (getRuntimeContext().getAttemptNumber() == 0
+                                        && currentIndex == TOTAL_ELEMENTS / 2) 
{
+                                    waitWhile(() -> 
completedCheckpointNum.get() <= 0);
+                                    throwArtificialFailure();
+                                }
+                            }
+                        }));
+    }
+
+    /** Recovery from checkpoint containing non-materialized state and 
materialized state. */
+    @Test
+    public void testMaterialization() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(10)
+                .enableChangelogStateBackend(true)
+                .getCheckpointConfig()
+                .setCheckpointStorage(TEMPORARY_FOLDER.newFolder().toURI());
+
+        
SerializableFunctionWithException<RunnableFuture<SnapshotResult<KeyedStateHandle>>>
+                snapshotResultConsumer =
+                        snapshotResultFuture -> {
+                            PENDING_MATERIALIZATION.add(snapshotResultFuture);
+                            return snapshotResultFuture;
+                        };
+        env.setStateBackend(
+                        new DelegatedStateBackendWrapper(
+                                delegatedStateBackend, snapshotResultConsumer))
+                .setRestartStrategy(RestartStrategies.fixedDelayRestart(2, 0));
+        env.configure(
+                new Configuration()
+                        .set(
+                                
StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL,
+                                Duration.ofMillis(10)));
+        waitAndAssert(
+                env,
+                buildStreamGraph(
+                        env,
+                        new ControlledSource() {
+                            @Override
+                            protected void beforeElement() throws Exception {
+                                if (getRuntimeContext().getAttemptNumber() == 0
+                                        && currentIndex == TOTAL_ELEMENTS / 4) 
{
+                                    waitWhile(
+                                            () ->
+                                                    
completedCheckpointNum.get() <= 0
+                                                            || 
PENDING_MATERIALIZATION.stream()
+                                                                    
.noneMatch(Future::isDone));
+                                    PENDING_MATERIALIZATION.clear();
+                                    throwArtificialFailure();
+                                } else if 
(getRuntimeContext().getAttemptNumber() == 1
+                                        && currentIndex == TOTAL_ELEMENTS / 2) 
{
+                                    waitWhile(
+                                            () ->
+                                                    
completedCheckpointNum.get() <= 1
+                                                            || 
PENDING_MATERIALIZATION.stream()
+                                                                    
.noneMatch(Future::isDone));
+                                    throwArtificialFailure();
+                                }
+                            }
+                        }));
+    }
+
+    @Test
+    public void testFailedMaterialization() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(10)
+                .enableChangelogStateBackend(true)
+                .getCheckpointConfig()
+                .setCheckpointStorage(TEMPORARY_FOLDER.newFolder().toURI());
+        env.setStateBackend(
+                        new DelegatedStateBackendWrapper(
+                                delegatedStateBackend,
+                                snapshotResultFuture -> {
+                                    
RunnableFuture<SnapshotResult<KeyedStateHandle>>
+                                            snapshotResultFutureWrapper =
+                                                    new 
FutureTaskWithException<>(
+                                                            () -> {
+                                                                
SnapshotResult<KeyedStateHandle>
+                                                                        
snapshotResult =
+                                                                               
 snapshotResultFuture
+                                                                               
         .get();
+                                                                if 
(PENDING_MATERIALIZATION.size()
+                                                                        == 0) {
+                                                                    throw new 
RuntimeException();
+                                                                } else {
+                                                                    return 
snapshotResult;
+                                                                }
+                                                            });
+                                    
PENDING_MATERIALIZATION.add(snapshotResultFutureWrapper);
+                                    return snapshotResultFutureWrapper;
+                                }))
+                .setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+        env.configure(
+                new Configuration()
+                        .set(
+                                
StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL,
+                                Duration.ofMillis(20))
+                        
.set(StateChangelogOptions.MATERIALIZATION_MAX_FAILURES_ALLOWED, 1));
+        waitAndAssert(
+                env,
+                buildStreamGraph(
+                        env,
+                        new ControlledSource() {
+                            @Override
+                            protected void beforeElement() throws Exception {
+                                waitWhile(
+                                        () ->
+                                                currentIndex >= TOTAL_ELEMENTS 
/ 2
+                                                        && 
PENDING_MATERIALIZATION.size() == 0);

Review comment:
       Thanks for clarifying. Shouldn't this check also include `isDone` part?
   
    (or whatever it will be replaced with as per comments above)




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