gaoyunhaii commented on a change in pull request #8:
URL: https://github.com/apache/flink-ml/pull/8#discussion_r731032333



##########
File path: 
flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/progresstrack/ProgressTracker.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.ml.iteration.progresstrack;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Tracks the epoch watermark from each input. Once the minimum epoch 
watermark changed, it would
+ * notify the listener.
+ */
+public class ProgressTracker {
+
+    private final ProgressTrackerListener progressTrackerListener;
+
+    private final List<InputStatus> inputStatuses;
+
+    private final LowerBoundMaintainer allInputsLowerBound;
+
+    public ProgressTracker(
+            int[] numberOfChannels, ProgressTrackerListener 
progressTrackerListener) {
+        checkState(numberOfChannels != null && numberOfChannels.length >= 1);
+        this.progressTrackerListener = checkNotNull(progressTrackerListener);
+
+        this.inputStatuses = new ArrayList<>(numberOfChannels.length);
+        for (int numberOfChannel : numberOfChannels) {
+            inputStatuses.add(new InputStatus(numberOfChannel));
+        }
+
+        this.allInputsLowerBound = new 
LowerBoundMaintainer(numberOfChannels.length);
+    }
+
+    public void onEpochWatermark(int inputIndex, String sender, int 
epochWatermark)
+            throws IOException {
+        InputStatus inputStatus = inputStatuses.get(inputIndex);
+        inputStatus.onUpdate(sender, epochWatermark);
+
+        if (inputStatus.getInputLowerBound() > 
allInputsLowerBound.getValue(inputIndex)) {
+            int oldLowerBound = allInputsLowerBound.getLowerBound();
+            allInputsLowerBound.updateValue(inputIndex, 
inputStatus.getInputLowerBound());
+            if (allInputsLowerBound.getLowerBound() > oldLowerBound) {
+                progressTrackerListener.onEpochWatermarkIncrement(
+                        allInputsLowerBound.getLowerBound());
+            }
+        }
+    }
+
+    @VisibleForTesting
+    int[] getNumberOfInputs() {
+        return inputStatuses.stream()
+                .mapToInt(inputStatus -> inputStatus.numberOfChannels)
+                .toArray();
+    }
+
+    private static class InputStatus {
+        private final int numberOfChannels;
+        private final Map<String, Integer> senderIndices;
+        private final LowerBoundMaintainer allChannelsLowerBound;
+
+        public InputStatus(int numberOfChannels) {
+            this.numberOfChannels = numberOfChannels;
+            this.senderIndices = new HashMap<>(numberOfChannels);
+            this.allChannelsLowerBound = new 
LowerBoundMaintainer(numberOfChannels);
+        }
+
+        public void onUpdate(String sender, int epochWatermark) {
+            int index = senderIndices.computeIfAbsent(sender, k -> 
senderIndices.size());
+            checkState(index < numberOfChannels);
+
+            allChannelsLowerBound.updateValue(index, epochWatermark);
+        }
+
+        public int getInputLowerBound() {
+            return allChannelsLowerBound.getLowerBound();
+        }
+    }
+
+    private static class LowerBoundMaintainer {
+
+        private final int[] values;

Review comment:
       I hold a bit different opinion: it might be not very possible for an 
algorithm to iterates Integer.MAX_VALUE times before convergence ? Using `long` 
would also bring overhead for record transmission through network. 
   
   We may revisit this decision after we met the actual scenarios. 




-- 
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-unsubscr...@flink.apache.org

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


Reply via email to