zhijiangW commented on a change in pull request #9717: [FLINK-14044] [runtime]
Reducing synchronization in AsyncWaitOperator
URL: https://github.com/apache/flink/pull/9717#discussion_r327976571
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/UnorderedStreamElementQueue.java
##########
@@ -19,192 +19,146 @@
package org.apache.flink.streaming.api.operators.async.queue;
import org.apache.flink.annotation.Internal;
-import org.apache.flink.streaming.api.operators.async.OperatorActions;
+import org.apache.flink.streaming.api.functions.async.ResultFuture;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayDeque;
-import java.util.Arrays;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.Deque;
import java.util.HashSet;
-import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.Queue;
import java.util.Set;
-import java.util.concurrent.Executor;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
/**
- * Unordered implementation of the {@link StreamElementQueue}. The unordered
stream element queue
- * emits asynchronous results as soon as they are completed. Additionally it
maintains the
- * watermark-stream record order. This means that no stream record can be
overtaken by a watermark
- * and no watermark can overtake a stream record. However, stream records
falling in the same
- * segment between two watermarks can overtake each other (their emission
order is not guaranteed).
+ * Unordered implementation of the {@link StreamElementQueue}. The unordered
stream element queue provides
+ * asynchronous results as soon as they are completed. Additionally it
maintains the watermark-stream record order.
+ * This means that no stream record can be overtaken by a watermark and no
watermark can overtake a stream record.
+ * However, stream records falling in the same segment between two watermarks
can overtake each other (their emission
+ * order is not guaranteed).
*/
@Internal
-public class UnorderedStreamElementQueue implements StreamElementQueue {
+public final class UnorderedStreamElementQueue<OUT> implements
StreamElementQueue<OUT> {
private static final Logger LOG =
LoggerFactory.getLogger(UnorderedStreamElementQueue.class);
/** Capacity of this queue. */
private final int capacity;
- /** Executor to run the onComplete callbacks. */
- private final Executor executor;
+ /** Queue of queue entries segmented by watermarks. */
+ private final Deque<Stage> stages;
- /** OperatorActions to signal the owning operator a failure. */
- private final OperatorActions operatorActions;
-
- /** Queue of uncompleted stream element queue entries segmented by
watermarks. */
- private final ArrayDeque<Set<StreamElementQueueEntry<?>>>
uncompletedQueue;
-
- /** Queue of completed stream element queue entries. */
- private final ArrayDeque<StreamElementQueueEntry<?>> completedQueue;
-
- /** First (chronologically oldest) uncompleted set of stream element
queue entries. */
- private Set<StreamElementQueueEntry<?>> firstSet;
-
- // Last (chronologically youngest) uncompleted set of stream element
queue entries. New
- // stream element queue entries are inserted into this set.
- private Set<StreamElementQueueEntry<?>> lastSet;
- private volatile int numberEntries;
-
- /** Locks and conditions for the blocking queue. */
- private final ReentrantLock lock;
- private final Condition notFull;
- private final Condition hasCompletedEntries;
-
- public UnorderedStreamElementQueue(
- int capacity,
- Executor executor,
- OperatorActions operatorActions) {
+ private int numberEntries;
+ public UnorderedStreamElementQueue(int capacity) {
Preconditions.checkArgument(capacity > 0, "The capacity must be
larger than 0.");
- this.capacity = capacity;
-
- this.executor = Preconditions.checkNotNull(executor,
"executor");
-
- this.operatorActions =
Preconditions.checkNotNull(operatorActions, "operatorActions");
-
- this.uncompletedQueue = new ArrayDeque<>(capacity);
- this.completedQueue = new ArrayDeque<>(capacity);
-
- this.firstSet = new HashSet<>(capacity);
- this.lastSet = firstSet;
+ this.capacity = capacity;
+ // most likely scenario are 4 stages <elements, watermark,
elements, watermark>
+ this.stages = new ArrayDeque<>(4);
this.numberEntries = 0;
-
- this.lock = new ReentrantLock();
- this.notFull = lock.newCondition();
- this.hasCompletedEntries = lock.newCondition();
}
@Override
- public <T> void put(StreamElementQueueEntry<T> streamElementQueueEntry)
throws InterruptedException {
- lock.lockInterruptibly();
-
- try {
- while (numberEntries >= capacity) {
- notFull.await();
+ public Optional<ResultFuture<OUT>> tryPut(StreamElement streamElement) {
+ if (size() < capacity) {
+ StreamElementQueueEntry<OUT> queueEntry;
+ if (streamElement.isRecord()) {
+ queueEntry = addRecord((StreamRecord<?>)
streamElement);
+ } else if (streamElement.isWatermark()) {
+ queueEntry = addWatermark((Watermark)
streamElement);
+ } else {
+ throw new UnsupportedOperationException("Cannot
enqueue " + streamElement);
}
- addEntry(streamElementQueueEntry);
- } finally {
- lock.unlock();
- }
- }
-
- @Override
- public <T> boolean tryPut(StreamElementQueueEntry<T>
streamElementQueueEntry) throws InterruptedException {
- lock.lockInterruptibly();
-
- try {
- if (numberEntries < capacity) {
- addEntry(streamElementQueueEntry);
+ numberEntries++;
- LOG.debug("Put element into unordered stream
element queue. New filling degree " +
- "({}/{}).", numberEntries, capacity);
+ LOG.debug("Put element into unordered stream element
queue. New filling degree " +
+ "({}/{}).", size(), capacity);
- return true;
- } else {
- LOG.debug("Failed to put element into unordered
stream element queue because it " +
- "was full ({}/{}).", numberEntries,
capacity);
+ return Optional.of(queueEntry);
+ } else {
+ LOG.debug("Failed to put element into unordered stream
element queue because it " +
+ "was full ({}/{}).", size(), capacity);
- return false;
- }
- } finally {
- lock.unlock();
+ return Optional.empty();
}
}
- @Override
- public AsyncResult peekBlockingly() throws InterruptedException {
- lock.lockInterruptibly();
+ private StreamElementQueueEntry<OUT> addRecord(StreamRecord<?> record) {
+ // ensure that there is at least one stage
+ Stage lastStage;
Review comment:
`Stage<OUT>`
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services