Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2629#discussion_r83382895
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBuffer.java
 ---
    @@ -0,0 +1,469 @@
    +/*
    + * 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.streaming.api.operators.async;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.streaming.api.datastream.AsyncDataStream;
    +import org.apache.flink.streaming.api.operators.Output;
    +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.ArrayList;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.concurrent.locks.Condition;
    +import java.util.concurrent.locks.Lock;
    +import java.util.concurrent.locks.ReentrantLock;
    +
    +/**
    + * AsyncCollectorBuffer will hold all {@link AsyncCollector} in its 
internal buffer,
    + * and emit results from {@link AsyncCollector} to the next operators 
following it by
    + * calling {@link Output#collect(Object)}
    + */
    +@Internal
    +public class AsyncCollectorBuffer<IN, OUT> {
    +   private static final Logger LOG = 
LoggerFactory.getLogger(AsyncCollectorBuffer.class);
    +
    +   /**
    +    * Max number of {@link AsyncCollector} in the buffer.
    +    */
    +   private int bufferSize;
    +
    +   private AsyncDataStream.OutputMode mode;
    +
    +   /**
    +    * {@link AsyncCollector} queue.
    +    */
    +   private SimpleLinkedList<AsyncCollector<IN, OUT>> queue = new 
SimpleLinkedList<>();
    +   /**
    +    * A hash map keeping {@link AsyncCollector} and their corresponding 
{@link StreamElement}
    +    */
    +   private Map<AsyncCollector<IN, OUT>, StreamElement> 
collectorToStreamElement = new HashMap<>();
    +   /**
    +    * A hash map keeping {@link AsyncCollector} and their node references 
in the #queue.
    +    */
    +   private Map<AsyncCollector<IN, OUT>, SimpleLinkedList.Node> 
collectorToQueue = new HashMap<>();
    +
    +   private LinkedList<AsyncCollector> finishedCollectors = new 
LinkedList<>();
    +
    +   /**
    +    * {@link TimestampedCollector} and {@link Output} to collect results 
and watermarks.
    +    */
    +   private TimestampedCollector<OUT> timestampedCollector;
    +   private Output<StreamRecord<OUT>> output;
    +
    +   /**
    +    * Locks and conditions to synchronize with main thread and emitter 
thread.
    +    */
    +   private Lock lock;
    +   private Condition notFull;
    +   private Condition taskDone;
    +   private Condition isEmpty;
    +
    +   /**
    +    * Error from user codes.
    +    */
    +   private volatile Exception error;
    +
    +   private Emitter emitter;
    +   private Thread emitThread;
    +
    +   private boolean isCheckpointing;
    +
    +   public AsyncCollectorBuffer(int maxSize, AsyncDataStream.OutputMode 
mode) {
    +           Preconditions.checkArgument(maxSize > 0, "Future buffer size 
should be greater than 0.");
    +
    +           this.bufferSize = maxSize;
    +           this.mode = mode;
    +
    +           this.lock = new ReentrantLock(true);
    +           this.notFull = this.lock.newCondition();
    +           this.taskDone = this.lock.newCondition();
    +           this.isEmpty = this.lock.newCondition();
    +
    +           this.emitter = new Emitter();
    +           this.emitThread = new Thread(emitter);
    +   }
    +
    +   /**
    +    * Add an {@link StreamRecord} into the buffer. A new {@link 
AsyncCollector} will be created and returned
    +    * corresponding to the input StreamRecord.
    +    * <p>
    +    * If buffer is full, caller will wait until new space is available.
    +    *
    +    * @param record StreamRecord
    +    * @return An AsyncCollector
    +    * @throws Exception InterruptedException or exceptions from 
AsyncCollector.
    +    */
    +   public AsyncCollector<IN, OUT> add(StreamRecord<IN> record) throws 
Exception {
    +           try {
    +                   lock.lock();
    +
    +                   notifyCheckpointDone();
    +
    +                   while (queue.size() >= bufferSize) {
    +                           notFull.await();
    +                   }
    +
    +                   // propagate error to the main thread
    +                   if (error != null) {
    +                           throw error;
    +                   }
    +
    +                   AsyncCollector<IN, OUT> collector = new 
AsyncCollector(this);
    +
    +                   collectorToQueue.put(collector, queue.add(collector));
    +                   collectorToStreamElement.put(collector, record);
    +
    +                   return collector;
    +           }
    +           finally {
    +                   lock.unlock();
    +           }
    +   }
    +
    +   /**
    +    * Add a {@link Watermark} into queue. A new AsyncCollector will be 
created and returned.
    +    * <p>
    +    * If queue is full, caller will be blocked here.
    +    *
    +    * @param watermark Watermark
    +    * @return AsyncCollector
    +    * @throws Exception Exceptions from async operation.
    +    */
    +   public AsyncCollector<IN, OUT> add(Watermark watermark) throws 
Exception {
    +           try {
    +                   lock.lock();
    +
    +                   notifyCheckpointDone();
    +
    +                   while (queue.size() >= bufferSize)
    +                           notFull.await();
    +
    +                   if (error != null) {
    +                           throw error;
    +                   }
    +
    +                   AsyncCollector<IN, OUT> collector = new 
AsyncCollector(this, true);
    +
    +                   collectorToQueue.put(collector, queue.add(collector));
    +                   collectorToStreamElement.put(collector, watermark);
    +
    +                   // signal emitter thread that current collector is ready
    +                   mark(collector);
    +
    +                   return collector;
    +           }
    +           finally {
    +                   lock.unlock();
    +           }
    +   }
    +
    +   /**
    +    * Notify the Emitter Thread that an AsyncCollector has completed.
    +    *
    +    * @param collector Completed AsyncCollector
    +    */
    +   void mark(AsyncCollector<IN, OUT> collector) {
    +           try {
    +                   lock.lock();
    +
    +                   if (mode == AsyncDataStream.OutputMode.UNORDERED) {
    +                           finishedCollectors.add(collector);
    +                   }
    +
    +                   taskDone.signal();
    +           }
    +           finally {
    +                   lock.unlock();
    +           }
    +   }
    +
    +   /**
    +    * Caller will wait here if buffer is not empty, meaning that not all 
async i/o tasks have returned yet.
    +    *
    +    * @throws Exception InterruptedException or Exceptions from 
AsyncCollector.
    +    */
    +   void waitEmpty() throws Exception {
    +           try {
    +                   lock.lock();
    +
    +                   notifyCheckpointDone();
    +
    +                   while (queue.size() != 0)
    +                           isEmpty.await();
    +
    +                   if (error != null) {
    +                           throw error;
    +                   }
    +           }
    +           finally {
    +                   lock.unlock();
    +           }
    +   }
    +
    +   public void startEmitterThread() {
    +           this.emitThread.start();
    +   }
    +
    +   public void stopEmitterThread() {
    +           if (emitter != null) {
    --- End diff --
    
    `emitter` and `emitThread` can probably never be `null`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to