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



##########
File path: 
flink-ml-iteration/src/main/java/org/apache/flink/iteration/Iterations.java
##########
@@ -112,15 +145,400 @@ public static DataStreamList 
iterateBoundedStreamsUntilTermination(
             ReplayableDataStreamList dataStreams,
             IterationConfig config,
             IterationBody body) {
-        Preconditions.checkArgument(
-                config.getOperatorLifeCycle() == 
IterationConfig.OperatorLifeCycle.ALL_ROUND);
-        
Preconditions.checkArgument(dataStreams.getReplayedDataStreams().size() == 0);
+        OperatorWrapper wrapper =
+                config.getOperatorLifeCycle() == 
IterationConfig.OperatorLifeCycle.ALL_ROUND
+                        ? new AllRoundOperatorWrapper<>()
+                        : new PerRoundOperatorWrapper<>();
 
-        return IterationFactory.createIteration(
+        List<DataStream<?>> allDatastreams = new ArrayList<>();
+        allDatastreams.addAll(dataStreams.getReplayedDataStreams());
+        allDatastreams.addAll(dataStreams.getNonReplayedStreams());
+
+        Set<Integer> replayedIndices =
+                IntStream.range(0, dataStreams.getReplayedDataStreams().size())
+                        .boxed()
+                        .collect(Collectors.toSet());
+
+        return createIteration(
                 initVariableStreams,
-                new DataStreamList(dataStreams.getNonReplayedStreams()),
+                new DataStreamList(allDatastreams),
+                replayedIndices,
                 body,
-                new AllRoundOperatorWrapper(),
+                wrapper,
                 true);
     }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    private static DataStreamList createIteration(
+            DataStreamList initVariableStreams,
+            DataStreamList dataStreams,
+            Set<Integer> replayedDataStreamIndices,
+            IterationBody body,
+            OperatorWrapper<?, IterationRecord<?>> initialOperatorWrapper,
+            boolean mayHaveCriteria) {
+        checkState(initVariableStreams.size() > 0, "There should be at least 
one variable stream");
+
+        IterationID iterationId = new IterationID();
+
+        List<TypeInformation<?>> initVariableTypeInfos = 
getTypeInfos(initVariableStreams);
+        List<TypeInformation<?>> dataStreamTypeInfos = 
getTypeInfos(dataStreams);
+
+        // Add heads and inputs
+        int totalInitVariableParallelism =
+                map(
+                                initVariableStreams,
+                                dataStream ->
+                                        dataStream.getParallelism() > 0
+                                                ? dataStream.getParallelism()
+                                                : dataStream
+                                                        
.getExecutionEnvironment()
+                                                        .getConfig()
+                                                        .getParallelism())
+                        .stream()
+                        .mapToInt(i -> i)
+                        .sum();
+        DataStreamList initVariableInputs = addInputs(initVariableStreams, 
false);
+        DataStreamList headStreams =
+                addHeads(
+                        initVariableStreams,
+                        initVariableInputs,
+                        iterationId,
+                        totalInitVariableParallelism,
+                        false,
+                        0);
+
+        DataStreamList dataStreamInputs = addInputs(dataStreams, true);
+        if (replayedDataStreamIndices.size() > 0) {
+            dataStreamInputs =
+                    addReplayer(
+                            headStreams.get(0),
+                            dataStreams,
+                            dataStreamInputs,
+                            replayedDataStreamIndices);
+        }
+
+        // Create the iteration body. We map the inputs of iteration body into 
the draft sources,

Review comment:
       Creates?




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