zhipeng93 commented on code in PR #248:
URL: https://github.com/apache/flink-ml/pull/248#discussion_r1293245279


##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/feedback/MpscQueue.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.feedback;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
+import org.apache.flink.statefun.flink.core.queue.Lock;
+import org.apache.flink.statefun.flink.core.queue.Locks;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Multi producers single consumer fifo queue.
+ *
+ * @param <T> The element type.
+ */
+@Internal
+public final class MpscQueue<T> implements Closeable {
+    private final Lock lock = Locks.spinLock();
+
+    private SpillableFeedbackQueue<T> activeQueue;
+    private SpillableFeedbackQueue<T> standByQueue;
+
+    private volatile boolean isClosed;
+
+    public MpscQueue(
+            IOManager ioManager,
+            MemoryManager memoryManager,
+            TypeSerializer<T> serializer,
+            long inMemoryBufferSize,
+            long pageSize)
+            throws MemoryAllocationException {
+        this.activeQueue =
+                new SpillableFeedbackQueue<>(
+                        ioManager, memoryManager, serializer, 
inMemoryBufferSize / 2, pageSize);
+        this.standByQueue =
+                new SpillableFeedbackQueue<>(
+                        ioManager, memoryManager, serializer, 
inMemoryBufferSize / 2, pageSize);
+    }
+
+    /**
+     * Adds an element to this (unbound) queue.
+     *
+     * @param element the element to add.
+     * @return the number of elements in the queue after the addition.
+     */
+    public long add(T element) {
+        Preconditions.checkState(element instanceof StreamRecord);
+
+        lock.lockUninterruptibly();
+        try {
+            SpillableFeedbackQueue<T> active = this.activeQueue;
+
+            active.add(element);
+            return active.size();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Atomically drains the queue.
+     *
+     * @return a batch of elements that obtained atomically from that queue.
+     */
+    public MutableObjectIterator<T> drainAll() {
+        lock.lockUninterruptibly();
+        try {
+            final SpillableFeedbackQueue<T> ready = this.activeQueue;
+            if (ready.size() == 0) {
+                return EmptyMutableObjectIterator.get();
+            }
+            this.activeQueue = this.standByQueue;
+            this.standByQueue = ready;
+            return ready.iterate();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    public void resetStandBy() throws Exception {

Review Comment:
   It seems confusing for users to call `resetStandBy`. How about we rename 
this method as `resetAfterDrainAll()`?



##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/feedback/SpillableFeedbackChannel.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.feedback;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.statefun.flink.core.feedback.FeedbackConsumer;
+import org.apache.flink.statefun.flink.core.feedback.SubtaskFeedbackKey;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Closeable;
+import java.util.Objects;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Single producer, single consumer channel, which can spill the records to 
disk when the in-memory
+ * buffer is full.
+ */
+@Internal
+public final class SpillableFeedbackChannel<T> implements Closeable {
+
+    /** The key that used to identify this channel. */
+    private final SubtaskFeedbackKey<T> key;
+
+    /** A single registered consumer. */
+    private final AtomicReference<ConsumerTask<T>> consumerRef = new 
AtomicReference<>();
+
+    /** The underlying queue used to hold the feedback results. */
+    private MpscQueue<T> queue;
+
+    /** Whether the feedback channel is initialized. */
+    private boolean initialized;
+
+    SpillableFeedbackChannel(SubtaskFeedbackKey<T> key) {
+        this.key = Objects.requireNonNull(key);
+    }
+
+    public void initialize(
+            IOManager ioManager,
+            MemoryManager memoryManager,
+            TypeSerializer<T> serializer,
+            long inMemoryBufferSize,
+            long pageSize)
+            throws MemoryAllocationException {
+        this.queue =
+                new MpscQueue<>(ioManager, memoryManager, serializer, 
inMemoryBufferSize, pageSize);
+        this.initialized = true;

Review Comment:
   Is there a case that this object is not initialized?



##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/feedback/MpscQueue.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.feedback;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
+import org.apache.flink.statefun.flink.core.queue.Lock;
+import org.apache.flink.statefun.flink.core.queue.Locks;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Multi producers single consumer fifo queue.
+ *
+ * @param <T> The element type.
+ */
+@Internal
+public final class MpscQueue<T> implements Closeable {
+    private final Lock lock = Locks.spinLock();
+
+    private SpillableFeedbackQueue<T> activeQueue;
+    private SpillableFeedbackQueue<T> standByQueue;
+
+    private volatile boolean isClosed;
+
+    public MpscQueue(
+            IOManager ioManager,
+            MemoryManager memoryManager,
+            TypeSerializer<T> serializer,
+            long inMemoryBufferSize,
+            long pageSize)
+            throws MemoryAllocationException {
+        this.activeQueue =
+                new SpillableFeedbackQueue<>(
+                        ioManager, memoryManager, serializer, 
inMemoryBufferSize / 2, pageSize);
+        this.standByQueue =
+                new SpillableFeedbackQueue<>(
+                        ioManager, memoryManager, serializer, 
inMemoryBufferSize / 2, pageSize);
+    }
+
+    /**
+     * Adds an element to this (unbound) queue.
+     *
+     * @param element the element to add.
+     * @return the number of elements in the queue after the addition.
+     */
+    public long add(T element) {
+        Preconditions.checkState(element instanceof StreamRecord);
+
+        lock.lockUninterruptibly();
+        try {
+            SpillableFeedbackQueue<T> active = this.activeQueue;
+
+            active.add(element);
+            return active.size();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Atomically drains the queue.
+     *
+     * @return a batch of elements that obtained atomically from that queue.
+     */
+    public MutableObjectIterator<T> drainAll() {
+        lock.lockUninterruptibly();
+        try {
+            final SpillableFeedbackQueue<T> ready = this.activeQueue;
+            if (ready.size() == 0) {
+                return EmptyMutableObjectIterator.get();
+            }
+            this.activeQueue = this.standByQueue;
+            this.standByQueue = ready;
+            return ready.iterate();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    public void resetStandBy() throws Exception {

Review Comment:
   nit: We may also make some of the methods in this PR package private.



##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/HeadOperator.java:
##########
@@ -498,6 +534,11 @@ public HeadOperatorStatus getStatus() {
         return status;
     }
 
+    @VisibleForTesting
+    public SpillableFeedbackChannel getFeedbackChannel() {

Review Comment:
   Can we remove this method and close the feedback channel in 
`HeadOperator#close()`?



##########
flink-ml-iteration/flink-ml-iteration-common/src/main/java/org/apache/flink/iteration/operator/feedback/MpscQueue.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.feedback;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
+import org.apache.flink.statefun.flink.core.queue.Lock;
+import org.apache.flink.statefun.flink.core.queue.Locks;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Multi producers single consumer fifo queue.
+ *
+ * @param <T> The element type.
+ */
+@Internal
+public final class MpscQueue<T> implements Closeable {
+    private final Lock lock = Locks.spinLock();
+
+    private SpillableFeedbackQueue<T> activeQueue;
+    private SpillableFeedbackQueue<T> standByQueue;
+
+    private volatile boolean isClosed;

Review Comment:
   Can you explain why do we need a `volatile isClosed` 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to