You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/26 03:11:36 UTC

[GitHub] [flink-ml] guoweiM commented on a change in pull request #14: [FLINK-8][iteration] Add per-round operator wrappers

guoweiM commented on a change in pull request #14:
URL: https://github.com/apache/flink-ml/pull/14#discussion_r735447020



##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/PerRoundOperatorWrapper.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.operator.OperatorWrapper;
+import org.apache.flink.iteration.proxy.ProxyKeySelector;
+import org.apache.flink.iteration.proxy.ProxyStreamPartitioner;
+import org.apache.flink.iteration.typeinfo.IterationRecordTypeInfo;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.util.OutputTag;
+
+/** The operator wrapper implementation for per-round wrappers. */
+public class PerRoundOperatorWrapper<T> implements OperatorWrapper<T, IterationRecord<T>> {
+
+    @Override
+    public StreamOperator<IterationRecord<T>> wrap(

Review comment:
       I think we should give more detailed exception:
   Tell the user that the iteration body could only build from the iteration variable and iteration constant.
   BTW, maybe we should also update the java doc.
   

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/AbstractPerRoundWrapperOperator.java
##########
@@ -0,0 +1,482 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.operator.AbstractWrapperOperator;
+import org.apache.flink.iteration.operator.OperatorUtils;
+import org.apache.flink.iteration.proxy.state.ProxyStateSnapshotContext;
+import org.apache.flink.iteration.proxy.state.ProxyStreamOperatorStateContext;
+import org.apache.flink.iteration.utils.ReflectionUtils;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateContext;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler;
+import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.util.LatencyStats;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+
+/** The base class for all the per-round wrapper operators. */
+public abstract class AbstractPerRoundWrapperOperator<T, S extends StreamOperator<T>>
+        extends AbstractWrapperOperator<T>
+        implements StreamOperatorStateHandler.CheckpointedStreamOperator {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(AbstractPerRoundWrapperOperator.class);
+
+    /** The wrapped operators for each round. */
+    private final Map<Integer, S> wrappedOperators;
+
+    protected final LatencyStats latencyStats;
+
+    private transient StreamOperatorStateContext streamOperatorStateContext;
+
+    private transient StreamOperatorStateHandler stateHandler;
+
+    private transient InternalTimeServiceManager<?> timeServiceManager;
+
+    private transient KeySelector<?, ?> stateKeySelector1;
+
+    private transient KeySelector<?, ?> stateKeySelector2;
+
+    public AbstractPerRoundWrapperOperator(
+            StreamOperatorParameters<IterationRecord<T>> parameters,
+            StreamOperatorFactory<T> operatorFactory) {
+        super(parameters, operatorFactory);
+
+        this.wrappedOperators = new HashMap<>();
+        this.latencyStats = initializeLatencyStats();
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    protected S getWrappedOperator(int round) {
+        S wrappedOperator = wrappedOperators.get(round);
+        if (wrappedOperator != null) {
+            return wrappedOperator;
+        }
+
+        // We needs to clone the operator factory to also support SimpleOperatorFactory.
+        try {
+            StreamOperatorFactory<T> clonedOperatorFactory =
+                    InstantiationUtil.clone(operatorFactory);
+            wrappedOperator =
+                    (S)
+                            StreamOperatorFactoryUtil.<T, S>createOperator(
+                                            clonedOperatorFactory,
+                                            (StreamTask) parameters.getContainingTask(),
+                                            parameters.getStreamConfig(),
+                                            proxyOutput,
+                                            parameters.getOperatorEventDispatcher())
+                                    .f0;
+            initializeStreamOperator(wrappedOperator, round);
+            wrappedOperators.put(round, wrappedOperator);
+            return wrappedOperator;
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+
+        return wrappedOperator;
+    }
+
+    @Override
+    public void onEpochWatermarkIncrement(int epochWatermark) throws IOException {
+        try {
+            // Deserts all the operators with round < epoch watermark. Notes that
+            // the onEpochWatermarkIncrement must be from 0 and increment by 1 each time.
+            if (wrappedOperators.containsKey(epochWatermark)) {
+                closeStreamOperator(wrappedOperators.get(epochWatermark), epochWatermark);
+                wrappedOperators.remove(epochWatermark);
+            }
+
+            super.onEpochWatermarkIncrement(epochWatermark);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+    }
+
+    protected void processForEachWrappedOperator(
+            BiConsumerWithException<Integer, S, Exception> consumer) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            consumer.accept(entry.getKey(), entry.getValue());
+        }
+    }
+
+    @Override
+    public void open() throws Exception {}
+
+    @Override
+    public void initializeState(StreamTaskStateInitializer streamTaskStateManager)
+            throws Exception {
+        final TypeSerializer<?> keySerializer =
+                streamConfig.getStateKeySerializer(containingTask.getUserCodeClassLoader());
+
+        streamOperatorStateContext =
+                streamTaskStateManager.streamOperatorStateContext(
+                        getOperatorID(),
+                        getClass().getSimpleName(),
+                        parameters.getProcessingTimeService(),
+                        this,
+                        keySerializer,
+                        containingTask.getCancelables(),
+                        metrics,
+                        streamConfig.getManagedMemoryFractionOperatorUseCaseOfSlot(
+                                ManagedMemoryUseCase.STATE_BACKEND,
+                                containingTask
+                                        .getEnvironment()
+                                        .getTaskManagerInfo()
+                                        .getConfiguration(),
+                                containingTask.getUserCodeClassLoader()),
+                        isUsingCustomRawKeyedState());
+
+        stateHandler =
+                new StreamOperatorStateHandler(
+                        streamOperatorStateContext,
+                        containingTask.getExecutionConfig(),
+                        containingTask.getCancelables());
+        stateHandler.initializeOperatorState(this);
+        this.timeServiceManager = streamOperatorStateContext.internalTimerServiceManager();
+
+        stateKeySelector1 =
+                streamConfig.getStatePartitioner(0, containingTask.getUserCodeClassLoader());
+        stateKeySelector2 =
+                streamConfig.getStatePartitioner(1, containingTask.getUserCodeClassLoader());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        // Do thing for now since we do not have states.
+    }
+
+    @Internal
+    protected boolean isUsingCustomRawKeyedState() {
+        return false;
+    }
+
+    @Override
+    public void finish() throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            closeStreamOperator(entry.getValue(), entry.getKey());
+        }
+        wrappedOperators.clear();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (stateHandler != null) {
+            stateHandler.dispose();
+        }
+    }
+
+    @Override
+    public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().prepareSnapshotPreBarrier(checkpointId);
+        }
+    }
+
+    @Override
+    public OperatorSnapshotFutures snapshotState(
+            long checkpointId,
+            long timestamp,
+            CheckpointOptions checkpointOptions,
+            CheckpointStreamFactory factory)
+            throws Exception {
+        return stateHandler.snapshotState(
+                this,
+                Optional.ofNullable(timeServiceManager),
+                streamConfig.getOperatorName(),
+                checkpointId,
+                timestamp,
+                checkpointOptions,
+                factory,
+                isUsingCustomRawKeyedState());
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            if (StreamOperatorStateHandler.CheckpointedStreamOperator.class.isAssignableFrom(
+                    entry.getValue().getClass())) {
+                ((StreamOperatorStateHandler.CheckpointedStreamOperator) entry.getValue())
+                        .snapshotState(new ProxyStateSnapshotContext(context));
+            }
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void setKeyContextElement1(StreamRecord record) throws Exception {
+        setKeyContextElement(record, stateKeySelector1);
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void setKeyContextElement2(StreamRecord record) throws Exception {
+        setKeyContextElement(record, stateKeySelector2);
+    }
+
+    private <T> void setKeyContextElement(StreamRecord<T> record, KeySelector<T, ?> selector)
+            throws Exception {
+        if (selector != null) {
+            Object key = selector.getKey(record.getValue());
+            setCurrentKey(key);
+        }
+    }
+
+    @Override
+    public OperatorMetricGroup getMetricGroup() {
+        return metrics;
+    }
+
+    @Override
+    public OperatorID getOperatorID() {
+        return streamConfig.getOperatorID();
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long l) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().notifyCheckpointComplete(l);
+        }
+    }
+
+    @Override
+    public void notifyCheckpointAborted(long checkpointId) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().notifyCheckpointAborted(checkpointId);
+        }
+    }
+
+    @Override
+    public void setCurrentKey(Object key) {
+        stateHandler.setCurrentKey(key);
+    }
+
+    @Override
+    public Object getCurrentKey() {
+        if (stateHandler == null) {
+            return null;
+        }
+
+        return stateHandler.getKeyedStateStore().orElse(null);
+    }
+
+    protected void reportOrForwardLatencyMarker(LatencyMarker marker) {
+        // all operators are tracking latencies
+        this.latencyStats.reportLatency(marker);
+
+        // everything except sinks forwards latency markers
+        this.output.emitLatencyMarker(marker);
+    }
+
+    private LatencyStats initializeLatencyStats() {

Review comment:
       I am not sure that the exact meaning of latency marker in the iteration. But it seems that both the wrapper and "internal" operators(per-round) have this metrics. So maybe only one of them have the metrics and report it is enough. And the code that initializing the `latencymarker` is the same as the internal implementation, which might introduce some burden in the future.
   
   IMHO an alternative way might be just report the limitation in the log. And after we could make it clear we add this back.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/AbstractPerRoundWrapperOperator.java
##########
@@ -0,0 +1,482 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.operator.AbstractWrapperOperator;
+import org.apache.flink.iteration.operator.OperatorUtils;
+import org.apache.flink.iteration.proxy.state.ProxyStateSnapshotContext;
+import org.apache.flink.iteration.proxy.state.ProxyStreamOperatorStateContext;
+import org.apache.flink.iteration.utils.ReflectionUtils;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateContext;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler;
+import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.util.LatencyStats;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+
+/** The base class for all the per-round wrapper operators. */
+public abstract class AbstractPerRoundWrapperOperator<T, S extends StreamOperator<T>>
+        extends AbstractWrapperOperator<T>
+        implements StreamOperatorStateHandler.CheckpointedStreamOperator {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(AbstractPerRoundWrapperOperator.class);
+
+    /** The wrapped operators for each round. */
+    private final Map<Integer, S> wrappedOperators;
+
+    protected final LatencyStats latencyStats;
+
+    private transient StreamOperatorStateContext streamOperatorStateContext;
+
+    private transient StreamOperatorStateHandler stateHandler;
+
+    private transient InternalTimeServiceManager<?> timeServiceManager;
+
+    private transient KeySelector<?, ?> stateKeySelector1;
+
+    private transient KeySelector<?, ?> stateKeySelector2;
+
+    public AbstractPerRoundWrapperOperator(
+            StreamOperatorParameters<IterationRecord<T>> parameters,
+            StreamOperatorFactory<T> operatorFactory) {
+        super(parameters, operatorFactory);
+
+        this.wrappedOperators = new HashMap<>();
+        this.latencyStats = initializeLatencyStats();
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    protected S getWrappedOperator(int round) {
+        S wrappedOperator = wrappedOperators.get(round);
+        if (wrappedOperator != null) {
+            return wrappedOperator;
+        }
+
+        // We needs to clone the operator factory to also support SimpleOperatorFactory.
+        try {
+            StreamOperatorFactory<T> clonedOperatorFactory =
+                    InstantiationUtil.clone(operatorFactory);
+            wrappedOperator =
+                    (S)
+                            StreamOperatorFactoryUtil.<T, S>createOperator(
+                                            clonedOperatorFactory,
+                                            (StreamTask) parameters.getContainingTask(),
+                                            parameters.getStreamConfig(),
+                                            proxyOutput,
+                                            parameters.getOperatorEventDispatcher())
+                                    .f0;
+            initializeStreamOperator(wrappedOperator, round);
+            wrappedOperators.put(round, wrappedOperator);
+            return wrappedOperator;
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+
+        return wrappedOperator;
+    }
+
+    @Override
+    public void onEpochWatermarkIncrement(int epochWatermark) throws IOException {
+        try {
+            // Deserts all the operators with round < epoch watermark. Notes that
+            // the onEpochWatermarkIncrement must be from 0 and increment by 1 each time.
+            if (wrappedOperators.containsKey(epochWatermark)) {
+                closeStreamOperator(wrappedOperators.get(epochWatermark), epochWatermark);
+                wrappedOperators.remove(epochWatermark);
+            }
+
+            super.onEpochWatermarkIncrement(epochWatermark);
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+    }
+
+    protected void processForEachWrappedOperator(
+            BiConsumerWithException<Integer, S, Exception> consumer) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            consumer.accept(entry.getKey(), entry.getValue());
+        }
+    }
+
+    @Override
+    public void open() throws Exception {}
+
+    @Override
+    public void initializeState(StreamTaskStateInitializer streamTaskStateManager)
+            throws Exception {
+        final TypeSerializer<?> keySerializer =
+                streamConfig.getStateKeySerializer(containingTask.getUserCodeClassLoader());
+
+        streamOperatorStateContext =
+                streamTaskStateManager.streamOperatorStateContext(
+                        getOperatorID(),
+                        getClass().getSimpleName(),
+                        parameters.getProcessingTimeService(),
+                        this,
+                        keySerializer,
+                        containingTask.getCancelables(),
+                        metrics,
+                        streamConfig.getManagedMemoryFractionOperatorUseCaseOfSlot(
+                                ManagedMemoryUseCase.STATE_BACKEND,
+                                containingTask
+                                        .getEnvironment()
+                                        .getTaskManagerInfo()
+                                        .getConfiguration(),
+                                containingTask.getUserCodeClassLoader()),
+                        isUsingCustomRawKeyedState());
+
+        stateHandler =
+                new StreamOperatorStateHandler(
+                        streamOperatorStateContext,
+                        containingTask.getExecutionConfig(),
+                        containingTask.getCancelables());
+        stateHandler.initializeOperatorState(this);
+        this.timeServiceManager = streamOperatorStateContext.internalTimerServiceManager();
+
+        stateKeySelector1 =
+                streamConfig.getStatePartitioner(0, containingTask.getUserCodeClassLoader());
+        stateKeySelector2 =
+                streamConfig.getStatePartitioner(1, containingTask.getUserCodeClassLoader());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        // Do thing for now since we do not have states.
+    }
+
+    @Internal
+    protected boolean isUsingCustomRawKeyedState() {
+        return false;
+    }
+
+    @Override
+    public void finish() throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            closeStreamOperator(entry.getValue(), entry.getKey());
+        }
+        wrappedOperators.clear();
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (stateHandler != null) {
+            stateHandler.dispose();
+        }
+    }
+
+    @Override
+    public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().prepareSnapshotPreBarrier(checkpointId);
+        }
+    }
+
+    @Override
+    public OperatorSnapshotFutures snapshotState(
+            long checkpointId,
+            long timestamp,
+            CheckpointOptions checkpointOptions,
+            CheckpointStreamFactory factory)
+            throws Exception {
+        return stateHandler.snapshotState(
+                this,
+                Optional.ofNullable(timeServiceManager),
+                streamConfig.getOperatorName(),
+                checkpointId,
+                timestamp,
+                checkpointOptions,
+                factory,
+                isUsingCustomRawKeyedState());
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            if (StreamOperatorStateHandler.CheckpointedStreamOperator.class.isAssignableFrom(
+                    entry.getValue().getClass())) {
+                ((StreamOperatorStateHandler.CheckpointedStreamOperator) entry.getValue())
+                        .snapshotState(new ProxyStateSnapshotContext(context));
+            }
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void setKeyContextElement1(StreamRecord record) throws Exception {
+        setKeyContextElement(record, stateKeySelector1);
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void setKeyContextElement2(StreamRecord record) throws Exception {
+        setKeyContextElement(record, stateKeySelector2);
+    }
+
+    private <T> void setKeyContextElement(StreamRecord<T> record, KeySelector<T, ?> selector)
+            throws Exception {
+        if (selector != null) {
+            Object key = selector.getKey(record.getValue());
+            setCurrentKey(key);
+        }
+    }
+
+    @Override
+    public OperatorMetricGroup getMetricGroup() {
+        return metrics;
+    }
+
+    @Override
+    public OperatorID getOperatorID() {
+        return streamConfig.getOperatorID();
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long l) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().notifyCheckpointComplete(l);
+        }
+    }
+
+    @Override
+    public void notifyCheckpointAborted(long checkpointId) throws Exception {
+        for (Map.Entry<Integer, S> entry : wrappedOperators.entrySet()) {
+            entry.getValue().notifyCheckpointAborted(checkpointId);
+        }
+    }
+
+    @Override
+    public void setCurrentKey(Object key) {
+        stateHandler.setCurrentKey(key);
+    }
+
+    @Override
+    public Object getCurrentKey() {
+        if (stateHandler == null) {
+            return null;
+        }
+
+        return stateHandler.getKeyedStateStore().orElse(null);
+    }
+
+    protected void reportOrForwardLatencyMarker(LatencyMarker marker) {
+        // all operators are tracking latencies
+        this.latencyStats.reportLatency(marker);
+
+        // everything except sinks forwards latency markers
+        this.output.emitLatencyMarker(marker);
+    }
+
+    private LatencyStats initializeLatencyStats() {
+        try {
+            Configuration taskManagerConfig =
+                    containingTask.getEnvironment().getTaskManagerInfo().getConfiguration();
+            int historySize = taskManagerConfig.getInteger(MetricOptions.LATENCY_HISTORY_SIZE);
+            if (historySize <= 0) {
+                LOG.warn(
+                        "{} has been set to a value equal or below 0: {}. Using default.",
+                        MetricOptions.LATENCY_HISTORY_SIZE,
+                        historySize);
+                historySize = MetricOptions.LATENCY_HISTORY_SIZE.defaultValue();
+            }
+
+            final String configuredGranularity =
+                    taskManagerConfig.getString(MetricOptions.LATENCY_SOURCE_GRANULARITY);
+            LatencyStats.Granularity granularity;
+            try {
+                granularity =
+                        LatencyStats.Granularity.valueOf(
+                                configuredGranularity.toUpperCase(Locale.ROOT));
+            } catch (IllegalArgumentException iae) {
+                granularity = LatencyStats.Granularity.OPERATOR;
+                LOG.warn(
+                        "Configured value {} option for {} is invalid. Defaulting to {}.",
+                        configuredGranularity,
+                        MetricOptions.LATENCY_SOURCE_GRANULARITY.key(),
+                        granularity);
+            }
+            MetricGroup jobMetricGroup = this.metrics.getJobMetricGroup();
+            return new LatencyStats(
+                    jobMetricGroup.addGroup("latency"),
+                    historySize,
+                    containingTask.getIndexInSubtaskGroup(),
+                    getOperatorID(),
+                    granularity);
+        } catch (Exception e) {
+            LOG.warn("An error occurred while instantiating latency metrics.", e);
+            return new LatencyStats(
+                    UnregisteredMetricGroups.createUnregisteredTaskManagerJobMetricGroup()
+                            .addGroup("latency"),
+                    1,
+                    0,
+                    new OperatorID(),
+                    LatencyStats.Granularity.SINGLE);
+        }
+    }
+
+    private void initializeStreamOperator(S operator, int round) throws Exception {
+        operator.initializeState(
+                (operatorID,
+                        operatorClassName,
+                        processingTimeService,
+                        keyContext,
+                        keySerializer,
+                        streamTaskCloseableRegistry,
+                        metricGroup,
+                        managedMemoryFraction,
+                        isUsingCustomRawKeyedState) ->
+                        new ProxyStreamOperatorStateContext(
+                                streamOperatorStateContext, getRoundStatePrefix(round)));
+        operator.open();
+    }
+
+    private void closeStreamOperator(S operator, int round) throws Exception {
+        OperatorUtils.processOperatorOrUdfIfSatisfy(
+                operator, BoundedOneInput.class, BoundedOneInput::endInput);
+        operator.finish();
+        operator.close();
+
+        // Cleanup the states used by this operator.
+        cleanupOperatorStates(round);
+
+        if (stateHandler.getKeyedStateBackend() != null) {
+            cleanupKeyedStates(round);
+        }
+    }
+
+    private void cleanupOperatorStates(int round) {
+        String roundPrefix = getRoundStatePrefix(round);
+        OperatorStateBackend operatorStateBackend = stateHandler.getOperatorStateBackend();
+
+        if (operatorStateBackend instanceof DefaultOperatorStateBackend) {
+            for (String fieldNames :
+                    new String[] {
+                        "registeredOperatorStates",
+                        "registeredBroadcastStates",
+                        "accessedStatesByName",
+                        "accessedBroadcastStatesByName"
+                    }) {
+                Map<String, ?> field =
+                        ReflectionUtils.getFieldValue(
+                                operatorStateBackend,
+                                DefaultOperatorStateBackend.class,
+                                fieldNames);
+                field.entrySet().removeIf(entry -> entry.getKey().startsWith(roundPrefix));
+            }
+        } else {
+            LOG.warn("Unable to cleanup the operator state {}", operatorStateBackend);
+        }
+    }
+
+    private void cleanupKeyedStates(int round) {
+        String roundPrefix = getRoundStatePrefix(round);

Review comment:
       I know that we should clean-up the per-round state. But it might be not so elegant that depending a specific implementation of the `statebackend`. Is there general way that could clean up the per-round's state.
   Or maybe you could open a jira to tracking this improvements.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/Iterations.java
##########
@@ -108,7 +108,7 @@ public static DataStreamList iterateUnboundedStreams(
     public static DataStreamList iterateBoundedStreamsUntilTermination(
             DataStreamList initVariableStreams, DataStreamList dataStreams, IterationBody body) {
         return IterationFactory.createIteration(
-            initVariableStreams, dataStreams, body, new AllRoundOperatorWrapper(), true);
+                initVariableStreams, dataStreams, body, new AllRoundOperatorWrapper(), true);

Review comment:
       Maybe this is the extra spaces?

##########
File path: flink-ml-iteration/pom.xml
##########
@@ -61,6 +61,14 @@ under the License.
             <scope>provided</scope>
         </dependency>
 
+        <!-- We have special treatment with the rocksdb state. -->

Review comment:
       Do you think that it is possible that we could only depend on the rocksdb in the testing phase?

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/MultipleInputPerRoundWrapperOperator.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Per-round wrapper for the multiple-inputs operator. */
+public class MultipleInputPerRoundWrapperOperator<OUT>
+        extends AbstractPerRoundWrapperOperator<OUT, MultipleInputStreamOperator<OUT>>
+        implements MultipleInputStreamOperator<IterationRecord<OUT>> {
+
+    public MultipleInputPerRoundWrapperOperator(
+            StreamOperatorParameters<IterationRecord<OUT>> parameters,
+            StreamOperatorFactory<OUT> operatorFactory) {
+        super(parameters, operatorFactory);
+    }
+
+    private <IN> void processElement(
+            int inputIndex,
+            Input<IN> input,
+            StreamRecord<IN> reusedInput,
+            StreamRecord<IterationRecord<IN>> element)
+            throws Exception {
+        switch (element.getValue().getType()) {
+            case RECORD:
+                reusedInput.replace(element.getValue().getValue(), element.getTimestamp());
+                setIterationContextRound(element.getValue().getEpoch());
+                input.processElement(reusedInput);
+                clearIterationContextRound();
+                break;
+            case EPOCH_WATERMARK:
+                onEpochWatermarkEvent(inputIndex, element.getValue());
+                break;
+            default:
+                throw new FlinkRuntimeException("Not supported iteration record type: " + element);
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"rawtypes"})
+    public List<Input> getInputs() {
+        List<Input> proxyInputs = new ArrayList<>();
+
+        // Determine how much inputs we have
+        List<StreamEdge> inEdges =
+                streamConfig.getInPhysicalEdges(containingTask.getUserCodeClassLoader());
+        int numberOfInputs =
+                inEdges.stream().map(StreamEdge::getTypeNumber).collect(Collectors.toSet()).size();
+
+        for (int i = 0; i < numberOfInputs; ++i) {
+            // TODO: Note that here we relies on the assumption that the

Review comment:
       Is there any way to assert this assumption?

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/AbstractPerRoundWrapperOperator.java
##########
@@ -0,0 +1,482 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.operator.AbstractWrapperOperator;
+import org.apache.flink.iteration.operator.OperatorUtils;
+import org.apache.flink.iteration.proxy.state.ProxyStateSnapshotContext;
+import org.apache.flink.iteration.proxy.state.ProxyStreamOperatorStateContext;
+import org.apache.flink.iteration.utils.ReflectionUtils;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateContext;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler;
+import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.util.LatencyStats;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+
+/** The base class for all the per-round wrapper operators. */
+public abstract class AbstractPerRoundWrapperOperator<T, S extends StreamOperator<T>>
+        extends AbstractWrapperOperator<T>
+        implements StreamOperatorStateHandler.CheckpointedStreamOperator {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(AbstractPerRoundWrapperOperator.class);
+
+    /** The wrapped operators for each round. */
+    private final Map<Integer, S> wrappedOperators;
+
+    protected final LatencyStats latencyStats;
+
+    private transient StreamOperatorStateContext streamOperatorStateContext;
+
+    private transient StreamOperatorStateHandler stateHandler;
+
+    private transient InternalTimeServiceManager<?> timeServiceManager;
+
+    private transient KeySelector<?, ?> stateKeySelector1;
+
+    private transient KeySelector<?, ?> stateKeySelector2;
+
+    public AbstractPerRoundWrapperOperator(
+            StreamOperatorParameters<IterationRecord<T>> parameters,
+            StreamOperatorFactory<T> operatorFactory) {
+        super(parameters, operatorFactory);
+
+        this.wrappedOperators = new HashMap<>();
+        this.latencyStats = initializeLatencyStats();
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    protected S getWrappedOperator(int round) {
+        S wrappedOperator = wrappedOperators.get(round);
+        if (wrappedOperator != null) {
+            return wrappedOperator;
+        }
+
+        // We needs to clone the operator factory to also support SimpleOperatorFactory.
+        try {
+            StreamOperatorFactory<T> clonedOperatorFactory =
+                    InstantiationUtil.clone(operatorFactory);
+            wrappedOperator =
+                    (S)
+                            StreamOperatorFactoryUtil.<T, S>createOperator(
+                                            clonedOperatorFactory,
+                                            (StreamTask) parameters.getContainingTask(),
+                                            parameters.getStreamConfig(),
+                                            proxyOutput,
+                                            parameters.getOperatorEventDispatcher())
+                                    .f0;
+            initializeStreamOperator(wrappedOperator, round);
+            wrappedOperators.put(round, wrappedOperator);
+            return wrappedOperator;
+        } catch (Exception e) {
+            ExceptionUtils.rethrow(e);
+        }
+
+        return wrappedOperator;
+    }
+
+    @Override
+    public void onEpochWatermarkIncrement(int epochWatermark) throws IOException {
+        try {
+            // Deserts all the operators with round < epoch watermark. Notes that

Review comment:
       Deserts? or Destroy.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/perround/AbstractPerRoundWrapperOperator.java
##########
@@ -0,0 +1,482 @@
+/*
+ * 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.iteration.operator.perround;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.operator.AbstractWrapperOperator;
+import org.apache.flink.iteration.operator.OperatorUtils;
+import org.apache.flink.iteration.proxy.state.ProxyStateSnapshotContext;
+import org.apache.flink.iteration.proxy.state.ProxyStreamOperatorStateContext;
+import org.apache.flink.iteration.utils.ReflectionUtils;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateContext;
+import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler;
+import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer;
+import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.util.LatencyStats;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+
+/** The base class for all the per-round wrapper operators. */
+public abstract class AbstractPerRoundWrapperOperator<T, S extends StreamOperator<T>>
+        extends AbstractWrapperOperator<T>
+        implements StreamOperatorStateHandler.CheckpointedStreamOperator {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(AbstractPerRoundWrapperOperator.class);
+
+    /** The wrapped operators for each round. */
+    private final Map<Integer, S> wrappedOperators;
+
+    protected final LatencyStats latencyStats;
+
+    private transient StreamOperatorStateContext streamOperatorStateContext;
+
+    private transient StreamOperatorStateHandler stateHandler;
+
+    private transient InternalTimeServiceManager<?> timeServiceManager;
+
+    private transient KeySelector<?, ?> stateKeySelector1;
+
+    private transient KeySelector<?, ?> stateKeySelector2;
+
+    public AbstractPerRoundWrapperOperator(
+            StreamOperatorParameters<IterationRecord<T>> parameters,
+            StreamOperatorFactory<T> operatorFactory) {
+        super(parameters, operatorFactory);
+
+        this.wrappedOperators = new HashMap<>();
+        this.latencyStats = initializeLatencyStats();
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    protected S getWrappedOperator(int round) {
+        S wrappedOperator = wrappedOperators.get(round);
+        if (wrappedOperator != null) {
+            return wrappedOperator;
+        }
+
+        // We needs to clone the operator factory to also support SimpleOperatorFactory.

Review comment:
       need?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org