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/02 16:21:14 UTC

[GitHub] [flink-ml] becketqin commented on a change in pull request #7: [FLINK-3][iteration] Implements the feedback mechanism and the controller operators.

becketqin commented on a change in pull request #7:
URL: https://github.com/apache/flink-ml/pull/7#discussion_r720580629



##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/IterationRecord.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.util.Objects;
+
+/** The wrapper for the records in iterative stream. */
+public class IterationRecord<T> implements Cloneable {
+
+    /** The type of mini-batch stream records. */
+    public enum Type {
+        RECORD,
+
+        EPOCH_WATERMARK,
+
+        BARRIER
+    }
+
+    private Type type;
+
+    private int round;

Review comment:
       Is this epoch? If so can we use the same terminology?

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/IterationRecord.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.util.Objects;
+
+/** The wrapper for the records in iterative stream. */
+public class IterationRecord<T> implements Cloneable {
+
+    /** The type of mini-batch stream records. */
+    public enum Type {
+        RECORD,
+
+        EPOCH_WATERMARK,
+
+        BARRIER
+    }
+
+    private Type type;
+
+    private int round;
+
+    // -------------------------- Fields for normal records -----------------
+
+    private T value;
+
+    // -------------------------- Fields for epoch watermark -----------------
+
+    private String sender;
+
+    // -------------------------- Fields for epoch watermark -----------------
+    private long checkpointId;
+
+    public static <T> IterationRecord<T> newRecord(T value, int round) {
+        return new IterationRecord<>(Type.RECORD, round, value, null, 0);
+    }
+
+    public static <T> IterationRecord<T> newEpochWatermark(int round, String sender) {
+        return new IterationRecord<>(Type.EPOCH_WATERMARK, round, null, sender, 0);
+    }
+
+    public static <T> IterationRecord<T> newBarrier(long checkpointId) {
+        return new IterationRecord<>(Type.BARRIER, 0, null, null, checkpointId);
+    }
+
+    private IterationRecord(Type type, int round, T value, String sender, long checkpointId) {
+        this.type = type;
+        this.round = round;
+        this.value = value;
+        this.sender = sender;
+        this.checkpointId = checkpointId;
+    }
+
+    public Type getType() {
+        return type;
+    }
+
+    public void setType(Type type) {
+        this.type = type;
+    }
+
+    public int getRound() {
+        return round;
+    }
+
+    public void setRound(int round) {
+        this.round = round;
+    }
+
+    public void incrementalRound() {

Review comment:
       incrementalRound -> incrementRound, or maybe just incRound.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperator.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.ml.iteration.operator.event.GloballyAlignedEvent;
+import org.apache.flink.ml.iteration.operator.event.SubtaskAlignedEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannel;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannelBroker;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackKey;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+
+/**
+ * The head operators unions the initialized variable stream and the feedback stream, and
+ * synchronize the epoch watermark (round).
+ */
+public class HeadOperator extends AbstractStreamOperator<IterationRecord<?>>
+        implements OneInputStreamOperator<IterationRecord<?>, IterationRecord<?>>,
+                FeedbackConsumer<StreamRecord<IterationRecord<?>>>,
+                OperatorEventHandler,
+                BoundedOneInput {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final OperatorEventGateway operatorEventGateway;
+
+    private final MailboxExecutor mailboxExecutor;
+
+    private final Map<Integer, Long> numFeedbackRecordsPerRound;
+
+    private transient BroadcastOutput<?> eventBroadcastOutput;
+
+    private transient StreamRecord<IterationRecord<?>> reusable;
+
+    private transient boolean shouldTerminate;
+
+    public HeadOperator(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            MailboxExecutor mailboxExecutor,
+            OperatorEventGateway operatorEventGateway,
+            ProcessingTimeService processingTimeService) {
+        this.iterationId = Objects.requireNonNull(iterationId);
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+        this.mailboxExecutor = Objects.requireNonNull(mailboxExecutor);
+        this.operatorEventGateway = Objects.requireNonNull(operatorEventGateway);
+        this.numFeedbackRecordsPerRound = new HashMap<>();
+
+        // Even though this operator does not use the processing
+        // time service, AbstractStreamOperator requires this
+        // field is non-null, otherwise we get a NullPointerException
+        super.processingTimeService = processingTimeService;
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<IterationRecord<?>>> output) {
+        super.setup(containingTask, config, output);
+        eventBroadcastOutput =
+                BroadcastOutputFactory.createBroadcastOutput(
+                        output, metrics.getIOMetricGroup().getNumRecordsOutCounter());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        reusable = new StreamRecord<>(null);
+
+        // Here we register a record
+        registerFeedbackConsumer(
+                (Runnable runnable) -> {
+                    if (!shouldTerminate) {
+                        mailboxExecutor.execute(runnable::run, "Head feedback");
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IterationRecord<?>> element) throws Exception {
+        processRecord(element);
+    }
+
+    @Override
+    public void processFeedback(StreamRecord<IterationRecord<?>> iterationRecord) throws Exception {
+        if (iterationRecord.getValue().getType() == IterationRecord.Type.RECORD) {
+            numFeedbackRecordsPerRound.compute(
+                    iterationRecord.getValue().getRound(),
+                    (round, count) -> count == null ? 1 : count + 1);
+        }
+        processRecord(iterationRecord);
+    }
+
+    private void processRecord(StreamRecord<IterationRecord<?>> iterationRecord) {
+        switch (iterationRecord.getValue().getType()) {
+            case RECORD:
+                reusable.replace(iterationRecord.getValue(), iterationRecord.getTimestamp());
+                output.collect(reusable);
+                break;
+            case EPOCH_WATERMARK:
+                LOG.info("Head Received epoch watermark {}", iterationRecord.getValue().getRound());
+                sendEpochWatermarkToCoordinator(iterationRecord.getValue().getRound());
+                break;
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void handleOperatorEvent(OperatorEvent operatorEvent) {
+        if (operatorEvent instanceof GloballyAlignedEvent) {
+            try {
+                GloballyAlignedEvent globallyAlignedEvent = (GloballyAlignedEvent) operatorEvent;
+                LOG.info("Received global event {}", globallyAlignedEvent);
+
+                shouldTerminate = globallyAlignedEvent.isTerminated();
+                reusable.replace(
+                        IterationRecord.newEpochWatermark(
+                                globallyAlignedEvent.isTerminated()
+                                        ? Integer.MAX_VALUE
+                                        : globallyAlignedEvent.getRound(),
+                                OperatorUtils.getUniqueSenderId(
+                                        getOperatorID(),
+                                        getRuntimeContext().getIndexOfThisSubtask())),
+                        0);

Review comment:
       Can we avoid the magic number 0 here? Also, if the timestamp is invalid, maybe we can use something like -1?

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperator.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.ml.iteration.operator.event.GloballyAlignedEvent;
+import org.apache.flink.ml.iteration.operator.event.SubtaskAlignedEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannel;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannelBroker;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackKey;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+
+/**
+ * The head operators unions the initialized variable stream and the feedback stream, and
+ * synchronize the epoch watermark (round).
+ */
+public class HeadOperator extends AbstractStreamOperator<IterationRecord<?>>
+        implements OneInputStreamOperator<IterationRecord<?>, IterationRecord<?>>,
+                FeedbackConsumer<StreamRecord<IterationRecord<?>>>,
+                OperatorEventHandler,
+                BoundedOneInput {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final OperatorEventGateway operatorEventGateway;
+
+    private final MailboxExecutor mailboxExecutor;
+
+    private final Map<Integer, Long> numFeedbackRecordsPerRound;
+
+    private transient BroadcastOutput<?> eventBroadcastOutput;
+
+    private transient StreamRecord<IterationRecord<?>> reusable;
+
+    private transient boolean shouldTerminate;
+
+    public HeadOperator(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            MailboxExecutor mailboxExecutor,
+            OperatorEventGateway operatorEventGateway,
+            ProcessingTimeService processingTimeService) {
+        this.iterationId = Objects.requireNonNull(iterationId);
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+        this.mailboxExecutor = Objects.requireNonNull(mailboxExecutor);
+        this.operatorEventGateway = Objects.requireNonNull(operatorEventGateway);
+        this.numFeedbackRecordsPerRound = new HashMap<>();
+
+        // Even though this operator does not use the processing
+        // time service, AbstractStreamOperator requires this
+        // field is non-null, otherwise we get a NullPointerException
+        super.processingTimeService = processingTimeService;
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<IterationRecord<?>>> output) {
+        super.setup(containingTask, config, output);
+        eventBroadcastOutput =
+                BroadcastOutputFactory.createBroadcastOutput(
+                        output, metrics.getIOMetricGroup().getNumRecordsOutCounter());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        reusable = new StreamRecord<>(null);
+
+        // Here we register a record
+        registerFeedbackConsumer(
+                (Runnable runnable) -> {
+                    if (!shouldTerminate) {
+                        mailboxExecutor.execute(runnable::run, "Head feedback");
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IterationRecord<?>> element) throws Exception {
+        processRecord(element);
+    }
+
+    @Override
+    public void processFeedback(StreamRecord<IterationRecord<?>> iterationRecord) throws Exception {
+        if (iterationRecord.getValue().getType() == IterationRecord.Type.RECORD) {
+            numFeedbackRecordsPerRound.compute(

Review comment:
       `numFeedbackRecordsPerRound` seems grow forever. We should remove the entries for the epoch that has been reported to the coordinator. 
   
   Also, currently the record counts here are only to determine whether the iteration should terminate or not. For that purpose, a set recording the epochs with records is sufficient. That may improve the performance a little bit.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperator.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.ml.iteration.operator.event.GloballyAlignedEvent;
+import org.apache.flink.ml.iteration.operator.event.SubtaskAlignedEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannel;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannelBroker;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackKey;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+
+/**
+ * The head operators unions the initialized variable stream and the feedback stream, and
+ * synchronize the epoch watermark (round).
+ */
+public class HeadOperator extends AbstractStreamOperator<IterationRecord<?>>
+        implements OneInputStreamOperator<IterationRecord<?>, IterationRecord<?>>,
+                FeedbackConsumer<StreamRecord<IterationRecord<?>>>,
+                OperatorEventHandler,
+                BoundedOneInput {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final OperatorEventGateway operatorEventGateway;
+
+    private final MailboxExecutor mailboxExecutor;
+
+    private final Map<Integer, Long> numFeedbackRecordsPerRound;
+
+    private transient BroadcastOutput<?> eventBroadcastOutput;
+
+    private transient StreamRecord<IterationRecord<?>> reusable;
+
+    private transient boolean shouldTerminate;
+
+    public HeadOperator(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            MailboxExecutor mailboxExecutor,
+            OperatorEventGateway operatorEventGateway,
+            ProcessingTimeService processingTimeService) {
+        this.iterationId = Objects.requireNonNull(iterationId);
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+        this.mailboxExecutor = Objects.requireNonNull(mailboxExecutor);
+        this.operatorEventGateway = Objects.requireNonNull(operatorEventGateway);
+        this.numFeedbackRecordsPerRound = new HashMap<>();
+
+        // Even though this operator does not use the processing
+        // time service, AbstractStreamOperator requires this
+        // field is non-null, otherwise we get a NullPointerException
+        super.processingTimeService = processingTimeService;
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<IterationRecord<?>>> output) {
+        super.setup(containingTask, config, output);
+        eventBroadcastOutput =
+                BroadcastOutputFactory.createBroadcastOutput(
+                        output, metrics.getIOMetricGroup().getNumRecordsOutCounter());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        reusable = new StreamRecord<>(null);
+
+        // Here we register a record
+        registerFeedbackConsumer(
+                (Runnable runnable) -> {
+                    if (!shouldTerminate) {
+                        mailboxExecutor.execute(runnable::run, "Head feedback");
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IterationRecord<?>> element) throws Exception {
+        processRecord(element);
+    }
+
+    @Override
+    public void processFeedback(StreamRecord<IterationRecord<?>> iterationRecord) throws Exception {
+        if (iterationRecord.getValue().getType() == IterationRecord.Type.RECORD) {
+            numFeedbackRecordsPerRound.compute(
+                    iterationRecord.getValue().getRound(),
+                    (round, count) -> count == null ? 1 : count + 1);
+        }
+        processRecord(iterationRecord);
+    }
+
+    private void processRecord(StreamRecord<IterationRecord<?>> iterationRecord) {
+        switch (iterationRecord.getValue().getType()) {
+            case RECORD:
+                reusable.replace(iterationRecord.getValue(), iterationRecord.getTimestamp());
+                output.collect(reusable);
+                break;
+            case EPOCH_WATERMARK:
+                LOG.info("Head Received epoch watermark {}", iterationRecord.getValue().getRound());
+                sendEpochWatermarkToCoordinator(iterationRecord.getValue().getRound());
+                break;
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void handleOperatorEvent(OperatorEvent operatorEvent) {
+        if (operatorEvent instanceof GloballyAlignedEvent) {
+            try {
+                GloballyAlignedEvent globallyAlignedEvent = (GloballyAlignedEvent) operatorEvent;
+                LOG.info("Received global event {}", globallyAlignedEvent);
+
+                shouldTerminate = globallyAlignedEvent.isTerminated();
+                reusable.replace(
+                        IterationRecord.newEpochWatermark(
+                                globallyAlignedEvent.isTerminated()
+                                        ? Integer.MAX_VALUE
+                                        : globallyAlignedEvent.getRound(),
+                                OperatorUtils.getUniqueSenderId(
+                                        getOperatorID(),
+                                        getRuntimeContext().getIndexOfThisSubtask())),
+                        0);
+                eventBroadcastOutput.broadcastEmit((StreamRecord) reusable);
+            } catch (Exception e) {
+                ExceptionUtils.rethrow(e);
+            }
+        }
+    }
+
+    @Override
+    public void endInput() throws Exception {
+        sendEpochWatermarkToCoordinator(0);

Review comment:
       We probably want to put a java doc here explaining why this is always zero, e.g. This is only invoked when there is a bounded initial variable streams, and it always has an epoch of zero. This will be the first watermark sent that let the iteration make progress.
   
   We probably also want to make it clear in the `Iterations` that the initial variable streams need to be bounded.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperator.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.ml.iteration.operator.event.GloballyAlignedEvent;
+import org.apache.flink.ml.iteration.operator.event.SubtaskAlignedEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannel;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannelBroker;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackKey;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+
+/**
+ * The head operators unions the initialized variable stream and the feedback stream, and
+ * synchronize the epoch watermark (round).
+ */
+public class HeadOperator extends AbstractStreamOperator<IterationRecord<?>>
+        implements OneInputStreamOperator<IterationRecord<?>, IterationRecord<?>>,
+                FeedbackConsumer<StreamRecord<IterationRecord<?>>>,
+                OperatorEventHandler,
+                BoundedOneInput {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final OperatorEventGateway operatorEventGateway;
+
+    private final MailboxExecutor mailboxExecutor;
+
+    private final Map<Integer, Long> numFeedbackRecordsPerRound;
+
+    private transient BroadcastOutput<?> eventBroadcastOutput;
+
+    private transient StreamRecord<IterationRecord<?>> reusable;
+
+    private transient boolean shouldTerminate;
+
+    public HeadOperator(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            MailboxExecutor mailboxExecutor,
+            OperatorEventGateway operatorEventGateway,
+            ProcessingTimeService processingTimeService) {
+        this.iterationId = Objects.requireNonNull(iterationId);
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+        this.mailboxExecutor = Objects.requireNonNull(mailboxExecutor);
+        this.operatorEventGateway = Objects.requireNonNull(operatorEventGateway);
+        this.numFeedbackRecordsPerRound = new HashMap<>();
+
+        // Even though this operator does not use the processing
+        // time service, AbstractStreamOperator requires this
+        // field is non-null, otherwise we get a NullPointerException
+        super.processingTimeService = processingTimeService;
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<IterationRecord<?>>> output) {
+        super.setup(containingTask, config, output);
+        eventBroadcastOutput =
+                BroadcastOutputFactory.createBroadcastOutput(
+                        output, metrics.getIOMetricGroup().getNumRecordsOutCounter());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        reusable = new StreamRecord<>(null);
+
+        // Here we register a record
+        registerFeedbackConsumer(
+                (Runnable runnable) -> {
+                    if (!shouldTerminate) {
+                        mailboxExecutor.execute(runnable::run, "Head feedback");
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IterationRecord<?>> element) throws Exception {
+        processRecord(element);
+    }
+
+    @Override
+    public void processFeedback(StreamRecord<IterationRecord<?>> iterationRecord) throws Exception {
+        if (iterationRecord.getValue().getType() == IterationRecord.Type.RECORD) {
+            numFeedbackRecordsPerRound.compute(
+                    iterationRecord.getValue().getRound(),
+                    (round, count) -> count == null ? 1 : count + 1);
+        }
+        processRecord(iterationRecord);
+    }
+
+    private void processRecord(StreamRecord<IterationRecord<?>> iterationRecord) {
+        switch (iterationRecord.getValue().getType()) {
+            case RECORD:
+                reusable.replace(iterationRecord.getValue(), iterationRecord.getTimestamp());
+                output.collect(reusable);
+                break;
+            case EPOCH_WATERMARK:
+                LOG.info("Head Received epoch watermark {}", iterationRecord.getValue().getRound());

Review comment:
       This logging could be quite verbose. Probably should be debug instead.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperator.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutput;
+import org.apache.flink.ml.iteration.broadcast.BroadcastOutputFactory;
+import org.apache.flink.ml.iteration.operator.event.GloballyAlignedEvent;
+import org.apache.flink.ml.iteration.operator.event.SubtaskAlignedEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannel;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackChannelBroker;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackKey;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.ExceptionUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+
+/**
+ * The head operators unions the initialized variable stream and the feedback stream, and
+ * synchronize the epoch watermark (round).
+ */
+public class HeadOperator extends AbstractStreamOperator<IterationRecord<?>>
+        implements OneInputStreamOperator<IterationRecord<?>, IterationRecord<?>>,
+                FeedbackConsumer<StreamRecord<IterationRecord<?>>>,
+                OperatorEventHandler,
+                BoundedOneInput {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final OperatorEventGateway operatorEventGateway;
+
+    private final MailboxExecutor mailboxExecutor;
+
+    private final Map<Integer, Long> numFeedbackRecordsPerRound;
+
+    private transient BroadcastOutput<?> eventBroadcastOutput;
+
+    private transient StreamRecord<IterationRecord<?>> reusable;
+
+    private transient boolean shouldTerminate;
+
+    public HeadOperator(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            MailboxExecutor mailboxExecutor,
+            OperatorEventGateway operatorEventGateway,
+            ProcessingTimeService processingTimeService) {
+        this.iterationId = Objects.requireNonNull(iterationId);
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+        this.mailboxExecutor = Objects.requireNonNull(mailboxExecutor);
+        this.operatorEventGateway = Objects.requireNonNull(operatorEventGateway);
+        this.numFeedbackRecordsPerRound = new HashMap<>();
+
+        // Even though this operator does not use the processing
+        // time service, AbstractStreamOperator requires this
+        // field is non-null, otherwise we get a NullPointerException
+        super.processingTimeService = processingTimeService;
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<IterationRecord<?>>> output) {
+        super.setup(containingTask, config, output);
+        eventBroadcastOutput =
+                BroadcastOutputFactory.createBroadcastOutput(
+                        output, metrics.getIOMetricGroup().getNumRecordsOutCounter());
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws Exception {
+        super.initializeState(context);
+
+        reusable = new StreamRecord<>(null);
+
+        // Here we register a record
+        registerFeedbackConsumer(
+                (Runnable runnable) -> {
+                    if (!shouldTerminate) {
+                        mailboxExecutor.execute(runnable::run, "Head feedback");
+                    }
+                });
+    }
+
+    @Override
+    public void processElement(StreamRecord<IterationRecord<?>> element) throws Exception {
+        processRecord(element);
+    }
+
+    @Override
+    public void processFeedback(StreamRecord<IterationRecord<?>> iterationRecord) throws Exception {
+        if (iterationRecord.getValue().getType() == IterationRecord.Type.RECORD) {
+            numFeedbackRecordsPerRound.compute(
+                    iterationRecord.getValue().getRound(),
+                    (round, count) -> count == null ? 1 : count + 1);
+        }
+        processRecord(iterationRecord);
+    }
+
+    private void processRecord(StreamRecord<IterationRecord<?>> iterationRecord) {
+        switch (iterationRecord.getValue().getType()) {
+            case RECORD:
+                reusable.replace(iterationRecord.getValue(), iterationRecord.getTimestamp());
+                output.collect(reusable);
+                break;
+            case EPOCH_WATERMARK:
+                LOG.info("Head Received epoch watermark {}", iterationRecord.getValue().getRound());
+                sendEpochWatermarkToCoordinator(iterationRecord.getValue().getRound());
+                break;
+        }
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void handleOperatorEvent(OperatorEvent operatorEvent) {
+        if (operatorEvent instanceof GloballyAlignedEvent) {
+            try {
+                GloballyAlignedEvent globallyAlignedEvent = (GloballyAlignedEvent) operatorEvent;
+                LOG.info("Received global event {}", globallyAlignedEvent);
+
+                shouldTerminate = globallyAlignedEvent.isTerminated();
+                reusable.replace(
+                        IterationRecord.newEpochWatermark(
+                                globallyAlignedEvent.isTerminated()
+                                        ? Integer.MAX_VALUE
+                                        : globallyAlignedEvent.getRound(),
+                                OperatorUtils.getUniqueSenderId(
+                                        getOperatorID(),
+                                        getRuntimeContext().getIndexOfThisSubtask())),
+                        0);
+                eventBroadcastOutput.broadcastEmit((StreamRecord) reusable);
+            } catch (Exception e) {
+                ExceptionUtils.rethrow(e);
+            }
+        }
+    }
+
+    @Override
+    public void endInput() throws Exception {
+        sendEpochWatermarkToCoordinator(0);
+        while (!shouldTerminate) {
+            mailboxExecutor.yield();
+        }
+    }
+
+    private void sendEpochWatermarkToCoordinator(int round) {
+        operatorEventGateway.sendEventToCoordinator(
+                new SubtaskAlignedEvent(
+                        round,
+                        numFeedbackRecordsPerRound.getOrDefault(round, 0L),
+                        isCriteriaStream));
+    }
+
+    private void registerFeedbackConsumer(Executor mailboxExecutor) {
+        int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+        int attemptNum = getRuntimeContext().getAttemptNumber();
+        FeedbackKey<StreamRecord<IterationRecord<?>>> feedbackKey =
+                new FeedbackKey<>(iterationId.toHexString(), feedbackIndex);

Review comment:
       Should probably use `OperatorUtils.createFeedbackKey()` instead.

##########
File path: flink-ml-iteration/src/main/java/org/apache/flink/ml/iteration/operator/HeadOperatorFactory.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.operator;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.ml.iteration.IterationID;
+import org.apache.flink.ml.iteration.IterationRecord;
+import org.apache.flink.ml.iteration.operator.coordinator.HeadOperatorCoordinator;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
+import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The Factory for the {@link HeadOperator}. */
+public class HeadOperatorFactory extends AbstractStreamOperatorFactory<IterationRecord<?>>
+        implements OneInputStreamOperatorFactory<IterationRecord<?>, IterationRecord<?>>,
+                CoordinatedOperatorFactory<IterationRecord<?>>,
+                YieldingOperatorFactory<IterationRecord<?>> {
+
+    private final IterationID iterationId;
+
+    private final int feedbackIndex;
+
+    private final boolean isCriteriaStream;
+
+    private final int totalInitialVariableParallelism;
+
+    private int criteriaStreamParallelism;
+
+    public HeadOperatorFactory(
+            IterationID iterationId,
+            int feedbackIndex,
+            boolean isCriteriaStream,
+            int totalInitialVariableParallelism) {
+        this.iterationId = iterationId;
+        this.feedbackIndex = feedbackIndex;
+        this.isCriteriaStream = isCriteriaStream;
+
+        checkArgument(
+                totalInitialVariableParallelism > 0,
+                "totalInitialVariableParallelism should be positive");
+        this.totalInitialVariableParallelism = totalInitialVariableParallelism;
+    }
+
+    public void setCriteriaStreamParallelism(int criteriaStreamParallelism) {
+        checkArgument(
+                criteriaStreamParallelism > 0,
+                "totalInitialVariableParallelism should be positive");
+        this.criteriaStreamParallelism = criteriaStreamParallelism;
+    }
+
+    @Override
+    public <T extends StreamOperator<IterationRecord<?>>> T createStreamOperator(
+            StreamOperatorParameters<IterationRecord<?>> streamOperatorParameters) {
+
+        // TODO: We would have to create a new mailboxExecutor since the given one
+        // is created with getChainedIndex as the priority, which seems to be a bug.

Review comment:
       Maybe refer to a Jira ticket number here.




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