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

 ##########
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandler.java
 ##########
 @@ -0,0 +1,434 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.DefaultKeyedStateStore;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyGroupsList;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateInitializationContextImpl;
+import org.apache.flink.runtime.state.StatePartitionStreamProvider;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.CloseableIterable;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * Class encapsulating various state backend handling logic for {@link 
StreamOperator} implementations.
+ */
+@PublicEvolving
+public class StreamOperatorStateHandler {
+
+       protected static final Logger LOG = 
LoggerFactory.getLogger(StreamOperatorStateHandler.class);
+
+       /** Backend for keyed state. This might be empty if we're not on a 
keyed stream. */
+       @Nullable
+       private final AbstractKeyedStateBackend<?> keyedStateBackend;
+       private final CloseableRegistry closeableRegistry;
+       @Nullable
+       private final DefaultKeyedStateStore keyedStateStore;
+       private final OperatorStateBackend operatorStateBackend;
+       private final InternalTimeServiceManager<?> timeServiceManager;
+       private final StreamOperatorStateContext context;
+
+       public StreamOperatorStateHandler(
+                       StreamOperatorStateContext context,
+                       ExecutionConfig executionConfig,
+                       CloseableRegistry closeableRegistry) {
+               this.context = context;
+               operatorStateBackend = context.operatorStateBackend();
+               keyedStateBackend = context.keyedStateBackend();
+               this.closeableRegistry = closeableRegistry;
+
+               if (keyedStateBackend != null) {
+                       keyedStateStore = new 
DefaultKeyedStateStore(keyedStateBackend, executionConfig);
+               }
+               else {
+                       keyedStateStore = null;
+               }
+
+               timeServiceManager = context.internalTimerServiceManager();
+       }
+
+       public void 
initializeOperatorState(ThrowingConsumer<StateInitializationContext, Exception> 
initializeOperatorAction) throws Exception {
+               CloseableIterable<KeyGroupStatePartitionStreamProvider> 
keyedStateInputs = context.rawKeyedStateInputs();
+               CloseableIterable<StatePartitionStreamProvider> 
operatorStateInputs = context.rawOperatorStateInputs();
+
+               try {
+                       StateInitializationContext initializationContext = new 
StateInitializationContextImpl(
+                               context.isRestored(), // information whether we 
restore or start for the first time
+                               operatorStateBackend, // access to operator 
state backend
+                               keyedStateStore, // access to keyed state 
backend
+                               keyedStateInputs, // access to keyed state 
stream
+                               operatorStateInputs); // access to operator 
state stream
+
+                       initializeOperatorAction.accept(initializationContext);
+               } finally {
+                       closeFromRegistry(operatorStateInputs, 
closeableRegistry);
+                       closeFromRegistry(keyedStateInputs, closeableRegistry);
+               }
+       }
+
+       private static void closeFromRegistry(Closeable closeable, 
CloseableRegistry registry) {
+               if (registry.unregisterCloseable(closeable)) {
+                       IOUtils.closeQuietly(closeable);
+               }
+       }
+
+       public void dispose() throws Exception {
+
+               Exception exception = null;
+
+               try {
+                       if 
(closeableRegistry.unregisterCloseable(operatorStateBackend)) {
+                               operatorStateBackend.close();
+                       }
+               } catch (Exception e) {
+                       exception = e;
+               }
+
+               try {
+                       if 
(closeableRegistry.unregisterCloseable(keyedStateBackend)) {
+                               keyedStateBackend.close();
+                       }
+               } catch (Exception e) {
+                       exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+               }
+
+               try {
+                       if (operatorStateBackend != null) {
+                               operatorStateBackend.dispose();
+                       }
+               } catch (Exception e) {
+                       exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+               }
+
+               try {
+                       if (keyedStateBackend != null) {
+                               keyedStateBackend.dispose();
+                       }
+               } catch (Exception e) {
+                       exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+               }
+
+               if (exception != null) {
+                       throw exception;
+               }
+       }
+
+       public OperatorSnapshotFutures snapshotState(
+                       ThrowingConsumer<StateSnapshotContext, Exception> 
snapshotStateAction,
+                       String operatorName,
+                       long checkpointId,
+                       long timestamp,
+                       CheckpointOptions checkpointOptions,
+                       CheckpointStreamFactory factory) throws 
CheckpointException {
+               KeyGroupRange keyGroupRange = null != keyedStateBackend ?
+                       keyedStateBackend.getKeyGroupRange() : 
KeyGroupRange.EMPTY_KEY_GROUP_RANGE;
+
+               OperatorSnapshotFutures snapshotInProgress = new 
OperatorSnapshotFutures();
+
+               StateSnapshotContextSynchronousImpl snapshotContext = new 
StateSnapshotContextSynchronousImpl(
+                       checkpointId,
+                       timestamp,
+                       factory,
+                       keyGroupRange,
+                       closeableRegistry);
+
+               return snapshotState(
+                       snapshotStateAction,
+                       operatorName,
+                       checkpointId,
+                       timestamp,
+                       checkpointOptions,
+                       factory,
+                       snapshotInProgress,
+                       snapshotContext);
+       }
+
+       @VisibleForTesting
+       OperatorSnapshotFutures snapshotState(
+                       ThrowingConsumer<StateSnapshotContext, Exception> 
snapshotStateAction,
+                       String operatorName,
+                       long checkpointId,
+                       long timestamp,
+                       CheckpointOptions checkpointOptions,
+                       CheckpointStreamFactory factory,
+                       OperatorSnapshotFutures snapshotInProgress,
+                       StateSnapshotContextSynchronousImpl snapshotContext) 
throws CheckpointException {
+               try {
+                       snapshotState(snapshotContext, operatorName);
+                       snapshotStateAction.accept(snapshotContext);
+
+                       
snapshotInProgress.setKeyedStateRawFuture(snapshotContext.getKeyedStateStreamFuture());
+                       
snapshotInProgress.setOperatorStateRawFuture(snapshotContext.getOperatorStateStreamFuture());
+
+                       if (null != operatorStateBackend) {
+                               
snapshotInProgress.setOperatorStateManagedFuture(
+                                       
operatorStateBackend.snapshot(checkpointId, timestamp, factory, 
checkpointOptions));
+                       }
+
+                       if (null != keyedStateBackend) {
+                               snapshotInProgress.setKeyedStateManagedFuture(
+                                       
keyedStateBackend.snapshot(checkpointId, timestamp, factory, 
checkpointOptions));
+                       }
+               } catch (Exception snapshotException) {
+                       try {
+                               snapshotInProgress.cancel();
+                       } catch (Exception e) {
+                               snapshotException.addSuppressed(e);
+                       }
+
+                       String snapshotFailMessage = "Could not complete 
snapshot " + checkpointId + " for operator " +
+                               operatorName + ".";
+
+                       try {
+                               snapshotContext.closeExceptionally();
+                       } catch (IOException e) {
+                               snapshotException.addSuppressed(e);
+                       }
+                       throw new CheckpointException(snapshotFailMessage, 
CheckpointFailureReason.CHECKPOINT_DECLINED, snapshotException);
+               }
+
+               return snapshotInProgress;
+       }
+
+       /**
+        * Stream operators with state, which want to participate in a snapshot 
need to override this hook method.
+        *
+        * @param context context that provides information and means required 
for taking a snapshot
+        * @param operatorName
+        */
+       public void snapshotState(StateSnapshotContext context, String 
operatorName) throws Exception {
+               final KeyedStateBackend<?> keyedStateBackend = 
getKeyedStateBackend();
+               //TODO all of this can be removed once heap-based timers are 
integrated with RocksDB incremental snapshots
+               if (keyedStateBackend instanceof AbstractKeyedStateBackend &&
+                       ((AbstractKeyedStateBackend<?>) 
keyedStateBackend).requiresLegacySynchronousTimerSnapshots()) {
+
+                       KeyedStateCheckpointOutputStream out;
+
+                       try {
+                               out = context.getRawKeyedOperatorStateOutput();
+                       } catch (Exception exception) {
+                               throw new Exception("Could not open raw keyed 
operator state stream for " +
+                                       operatorName + '.', exception);
+                       }
+
+                       try {
+                               KeyGroupsList allKeyGroups = 
out.getKeyGroupList();
+                               for (int keyGroupIdx : allKeyGroups) {
+                                       out.startNewKeyGroup(keyGroupIdx);
+
+                                       
timeServiceManager.snapshotStateForKeyGroup(
+                                               new 
DataOutputViewStreamWrapper(out), keyGroupIdx);
+                               }
+                       } catch (Exception exception) {
+                               throw new Exception("Could not write timer 
service of " + operatorName +
+                                       " to checkpoint state stream.", 
exception);
+                       } finally {
+                               try {
+                                       out.close();
+                               } catch (Exception closeException) {
+                                       LOG.warn("Could not close raw keyed 
operator state stream for {}. This " +
+                                               "might have prevented deleting 
some state data.", operatorName, closeException);
+                               }
+                       }
+               }
+       }
+
+       public void notifyCheckpointComplete(long checkpointId) throws 
Exception {
+               if (keyedStateBackend != null) {
+                       
keyedStateBackend.notifyCheckpointComplete(checkpointId);
+               }
+       }
+
+       @SuppressWarnings("unchecked")
+       public <K> KeyedStateBackend<K> getKeyedStateBackend() {
+               return (KeyedStateBackend<K>) keyedStateBackend;
+       }
 
 Review comment:
   I think we should parameterize new classes (`AbstractStreamOperatorV2` and 
`StreamOperatorStateHandler`) and keep the old one (`AbstractStreamOperator`) 
as is.
   
   @pnowojski , I didn't get your point here:
   > KeyedOp and NonKeyedOp won't work nicely, unless you want to multiply 
number of abstract classes that are extending from 
AbstractStreamOperator/AbstractStreamOperatorV2 by factor of two.
   
   Can you please explain if it's still relevant?

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to