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/21 06:44:58 UTC

[GitHub] [flink-ml] guoweiM commented on a change in pull request #8: [FLINK-4][iteration] Add operator wrapper for all-round iterations.

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



##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/progresstrack/OperatorEpochWatermarkTracker.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.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 OperatorEpochWatermarkTracker {
+
+    private final OperatorEpochWatermarkTrackerListener progressTrackerListener;
+
+    private final List<InputStatus> inputStatuses;
+
+    private final LowerBoundMaintainer allInputsLowerBound;
+
+    public OperatorEpochWatermarkTracker(
+            int[] numberOfChannels, OperatorEpochWatermarkTrackerListener 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() {

Review comment:
       I am not very sure but maybe we could remove this method, which just test the internal implementation.
   What is in my mind is that we could test the `OperatorEpochWatermarkTrackerListener`.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/EpochAware.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+import java.util.function.Supplier;
+
+/**

Review comment:
       I would suggest to describe the difference between this interface and `IterationListener` more in the document, and why this can't be put in the original Udf/operator's `Context`. 
   I think this might be better to let user understand the semantics of this interface.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/operator/AbstractWrapperOperator.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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;
+
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.iteration.progresstrack.OperatorEpochWatermarkTracker;
+import org.apache.flink.iteration.progresstrack.OperatorEpochWatermarkTrackerFactory;
+import org.apache.flink.iteration.progresstrack.OperatorEpochWatermarkTrackerListener;
+import org.apache.flink.iteration.proxy.ProxyOutput;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.metrics.groups.InternalOperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.Output;
+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.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The base class of all the wrapper operators. It provides the alignment functionality. */
+public abstract class AbstractWrapperOperator<T>
+        implements StreamOperator<IterationRecord<T>>, OperatorEpochWatermarkTrackerListener {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractWrapperOperator.class);
+
+    protected final StreamOperatorParameters<IterationRecord<T>> parameters;
+
+    protected final StreamConfig streamConfig;
+
+    protected final StreamTask<?, ?> containingTask;
+
+    protected final Output<StreamRecord<IterationRecord<T>>> output;
+
+    protected final StreamOperatorFactory<T> operatorFactory;
+
+    // --------------- proxy ---------------------------
+
+    protected final ProxyOutput<T> proxyOutput;
+
+    protected final EpochSupplier epochWatermarkSupplier;
+
+    // --------------- Metrics ---------------------------
+
+    /** Metric group for the operator. */
+    protected final InternalOperatorMetricGroup metrics;
+
+    // ------------- Iteration Related --------------------
+
+    protected final OperatorEpochWatermarkTracker epochWatermarkTracker;
+
+    protected final String uniqueSenderId;

Review comment:
       Why would we use the string as a id here ? 

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/iteration/proxy/ProxyOutput.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.proxy;
+
+import org.apache.flink.iteration.IterationRecord;
+import org.apache.flink.iteration.typeinfo.IterationRecordTypeInfo;
+import org.apache.flink.streaming.api.operators.Output;
+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.OutputTag;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/** Proxy output to provide to the wrapped operator. */
+public class ProxyOutput<T> implements Output<StreamRecord<T>> {
+
+    private final Output<StreamRecord<IterationRecord<T>>> output;
+
+    private final StreamRecord<IterationRecord<T>> reuseRecord;
+
+    private final Map<String, SideOutputCache> sideOutputCaches = new HashMap<>();
+
+    private Integer contextRound;
+
+    public ProxyOutput(Output<StreamRecord<IterationRecord<T>>> output) {
+        this.output = Objects.requireNonNull(output);
+        this.reuseRecord = new StreamRecord<>(IterationRecord.newRecord(null, 0));
+    }
+
+    public void setContextRound(Integer contextRound) {
+        this.contextRound = contextRound;
+    }
+
+    @Override
+    public void emitWatermark(Watermark mark) {
+        output.emitWatermark(mark);
+    }
+
+    @Override
+    public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {
+        output.emitWatermarkStatus(watermarkStatus);
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {
+        SideOutputCache sideOutputCache =
+                sideOutputCaches.computeIfAbsent(
+                        outputTag.getId(),
+                        (ignored) ->
+                                new SideOutputCache(
+                                        new OutputTag<IterationRecord<?>>(
+                                                outputTag.getId(),
+                                                new IterationRecordTypeInfo(
+                                                        outputTag.getTypeInfo())),
+                                        new StreamRecord<>(IterationRecord.newRecord(null, 0))));
+        sideOutputCache.cachedRecord.replace(
+                IterationRecord.newRecord(record.getValue(), contextRound), record.getTimestamp());
+        output.collect(sideOutputCache.tag, sideOutputCache.cachedRecord);
+    }
+
+    @Override
+    public void emitLatencyMarker(LatencyMarker latencyMarker) {
+        output.emitLatencyMarker(latencyMarker);
+    }
+
+    @Override
+    public void collect(StreamRecord<T> tStreamRecord) {

Review comment:
       tStreamRecord or streamRecord?




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