AHeise commented on a change in pull request #11403: [FLINK-16316][operators] 
Implement new StreamOperatorBase as a replacement for AbstractStreamOperator
URL: https://github.com/apache/flink/pull/11403#discussion_r392907594
 
 

 ##########
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandlerTest.java
 ##########
 @@ -0,0 +1,207 @@
+/*
+ * 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.api.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.util.InterceptingOperatorMetricGroup;
+import org.apache.flink.runtime.operators.testutils.ExpectedTestException;
+import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
+import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.junit.Test;
+
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link StreamOperatorStateHandlerTest}.
+ */
+public class StreamOperatorStateHandlerTest {
+       /**
+        * Tests that a failing snapshot method call to the keyed state backend 
will trigger the closing
+        * of the StateSnapshotContextSynchronousImpl and the cancellation of 
the
+        * OperatorSnapshotResult. The latter is supposed to also cancel all 
assigned futures.
+        */
+       @Test
+       public void testFailingBackendSnapshotMethod() throws Exception {
+               final long checkpointId = 42L;
+               final long timestamp = 1L;
+
+               final CloseableRegistry closeableRegistry = new 
CloseableRegistry();
+
+               RunnableFuture<SnapshotResult<KeyedStateHandle>> 
keyedStateManagedFuture = new CancelableFuture<>();
+               RunnableFuture<SnapshotResult<KeyedStateHandle>> 
keyedStateRawFuture = new CancelableFuture<>();
+               RunnableFuture<SnapshotResult<OperatorStateHandle>> 
operatorStateManagedFuture = new CancelableFuture<>();
+               RunnableFuture<SnapshotResult<OperatorStateHandle>> 
operatorStateRawFuture = new CancelableFuture<>();
+
+               OperatorSnapshotFutures operatorSnapshotResult = new 
OperatorSnapshotFutures(
+                       keyedStateManagedFuture,
+                       keyedStateRawFuture,
+                       operatorStateManagedFuture,
+                       operatorStateRawFuture);
+
+               StateSnapshotContextSynchronousImpl context = new 
TestStateSnapshotContextSynchronousImpl(checkpointId, timestamp, 
closeableRegistry);
+               context.getRawKeyedOperatorStateOutput();
+               context.getRawOperatorStateOutput();
+
+               StreamTaskStateInitializerImpl stateInitializer =
+                       new StreamTaskStateInitializerImpl(new 
MockEnvironmentBuilder().build(), new MemoryStateBackend());
+               StreamOperatorStateContext stateContext = 
stateInitializer.streamOperatorStateContext(
+                       new OperatorID(),
+                       "whatever",
+                       new TestProcessingTimeService(),
+                       new UnUsedKeyContext(),
+                       IntSerializer.INSTANCE,
+                       closeableRegistry,
+                       new InterceptingOperatorMetricGroup());
+               StreamOperatorStateHandler stateHandler = new 
StreamOperatorStateHandler(stateContext, new ExecutionConfig(), 
closeableRegistry);
+
+               final String keyedStateField = "keyedStateField";
+               final String operatorStateField = "operatorStateField";
+
+               stateHandler.setCurrentKey(44L);
+               stateHandler.initializeOperatorState(stateSnapshotContext -> {
+                       stateSnapshotContext.getKeyedStateStore()
+                               .getState(new 
ValueStateDescriptor<>(keyedStateField, LongSerializer.INSTANCE))
+                               .update(42L);
+                       stateSnapshotContext.getOperatorStateStore()
+                               .getListState(new 
ListStateDescriptor<>(operatorStateField, LongSerializer.INSTANCE))
+                               .add(42L);
+               });
+
+               
assertThat(stateContext.operatorStateBackend().getRegisteredStateNames(), 
is(not(empty())));
+               
assertThat(stateContext.keyedStateBackend().numKeyValueStatesByName(), 
equalTo(1));
+
+               try {
+                       stateHandler.snapshotState(
+                               stateSnapshotContext -> {
+                                       throw new ExpectedTestException();
+                               },
+                               "42",
+                               42,
+                               42,
+                               
CheckpointOptions.forCheckpointWithDefaultLocation(),
+                               new MemCheckpointStreamFactory(1024),
+                               operatorSnapshotResult,
+                               context);
+                       fail("Exception expected.");
+               } catch (CheckpointException e) {
+                       // We can not check for ExpectedTestException class 
directly,
+                       // as CheckpointException is wrapping the cause with 
SerializedThrowable
+                       if (!ExceptionUtils.findThrowableWithMessage(e, 
ExpectedTestException.MESSAGE).isPresent()) {
+                               throw e;
+                       }
+               }
+
+               assertTrue(keyedStateManagedFuture.isCancelled());
+               assertTrue(keyedStateRawFuture.isCancelled());
+               assertTrue(context.getKeyedStateStreamFuture().isCancelled());
+               assertTrue(operatorStateManagedFuture.isCancelled());
+               assertTrue(operatorStateRawFuture.isCancelled());
+               
assertTrue(context.getOperatorStateStreamFuture().isCancelled());
+
+               stateHandler.dispose();
+
+               
assertThat(stateContext.operatorStateBackend().getRegisteredBroadcastStateNames(),
 is(empty()));
+               
assertThat(stateContext.operatorStateBackend().getRegisteredStateNames(), 
is(empty()));
+               
assertThat(stateContext.keyedStateBackend().numKeyValueStatesByName(), is(0));
+       }
+
+       private static class TestStateSnapshotContextSynchronousImpl extends 
StateSnapshotContextSynchronousImpl {
+               public TestStateSnapshotContextSynchronousImpl(
+                               long checkpointId,
+                               long timestamp,
+                               CloseableRegistry closeableRegistry) {
+                       super(checkpointId, timestamp, new 
MemCheckpointStreamFactory(1024), new KeyGroupRange(0, 2), closeableRegistry);
+                       this.keyedStateCheckpointClosingFuture = new 
CancelableFuture<>();
+                       this.operatorStateCheckpointClosingFuture = new 
CancelableFuture<>();
+               }
+       }
+
+       private static class CancelableFuture<T> implements RunnableFuture<T> {
 
 Review comment:
   Could we use FutureTask instead of implementing it ourselves?

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


With regards,
Apache Git Services

Reply via email to