Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id BB522200BC3 for ; Thu, 3 Nov 2016 16:31:12 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B9FE9160AFF; Thu, 3 Nov 2016 15:31:12 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B853B160B0D for ; Thu, 3 Nov 2016 16:31:11 +0100 (CET) Received: (qmail 65233 invoked by uid 500); 3 Nov 2016 15:31:10 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 65106 invoked by uid 99); 3 Nov 2016 15:31:10 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Nov 2016 15:31:10 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 678721A8C5B for ; Thu, 3 Nov 2016 15:31:10 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -7.018 X-Spam-Level: X-Spam-Status: No, score=-7.018 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-2.999, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id TESbNNNSwG2n for ; Thu, 3 Nov 2016 15:31:07 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id AD68F5FC8D for ; Thu, 3 Nov 2016 15:31:01 +0000 (UTC) Received: (qmail 54828 invoked by uid 99); 3 Nov 2016 15:31:00 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Nov 2016 15:31:00 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 80F4CDFE8F; Thu, 3 Nov 2016 15:31:00 +0000 (UTC) From: tillrohrmann To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #2629: [FLINK-4391] Provide support for asynchronous oper... Content-Type: text/plain Message-Id: <20161103153100.80F4CDFE8F@git1-us-west.apache.org> Date: Thu, 3 Nov 2016 15:31:00 +0000 (UTC) archived-at: Thu, 03 Nov 2016 15:31:12 -0000 Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2629#discussion_r86361495 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBuffer.java --- @@ -0,0 +1,494 @@ +/* + * 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.LatencyMarker; +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 { + private static final Logger LOG = LoggerFactory.getLogger(AsyncCollectorBuffer.class); + + /** + * Max number of {@link AsyncCollector} in the buffer. + */ + private final int bufferSize; + + private final AsyncDataStream.OutputMode mode; + + private final AsyncWaitOperator operator; + + /** + * {@link AsyncCollector} queue. + */ + private final SimpleLinkedList> queue = new SimpleLinkedList<>(); + /** + * A hash map keeping {@link AsyncCollector} and their corresponding {@link StreamElement} + */ + private final Map, StreamElement> collectorToStreamElement = new HashMap<>(); + /** + * A hash map keeping {@link AsyncCollector} and their node references in the #queue. + */ + private final Map, SimpleLinkedList.Node> collectorToQueue = new HashMap<>(); + + private final LinkedList finishedCollectors = new LinkedList<>(); + + /** + * {@link TimestampedCollector} and {@link Output} to collect results and watermarks. + */ + private TimestampedCollector timestampedCollector; + private Output> output; + + /** + * Locks and conditions to synchronize with main thread and emitter thread. + */ + private final Lock lock; + private final Condition notFull; + private final Condition taskDone; + private final Condition isEmpty; + + /** + * Error from user codes. + */ + private volatile Exception error; + + private final Emitter emitter; + private final Thread emitThread; + + private boolean isCheckpointing; + + public AsyncCollectorBuffer(int maxSize, AsyncDataStream.OutputMode mode, AsyncWaitOperator operator) { + Preconditions.checkArgument(maxSize > 0, "Future buffer size should be greater than 0."); + + this.bufferSize = maxSize; + this.mode = mode; + this.operator = operator; + + 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. + *

+ * 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 add(StreamRecord 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 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. + *

+ * If queue is full, caller will be blocked here. + * + * @param watermark Watermark + * @return AsyncCollector + * @throws Exception Exceptions from async operation. + */ + public AsyncCollector add(Watermark watermark) throws Exception { + return processMark(watermark); + } + + /** + * Add a {@link LatencyMarker} into queue. A new AsyncCollector will be created and returned. + *

+ * If queue is full, caller will be blocked here. + * + * @param latencyMarker LatencyMarker + * @return AsyncCollector + * @throws Exception Exceptions from async operation. + */ + public AsyncCollector add(LatencyMarker latencyMarker) throws Exception { + return processMark(latencyMarker); + } + + private AsyncCollector processMark(StreamElement mark) throws Exception { + try { + lock.lock(); + + notifyCheckpointDone(); + + while (queue.size() >= bufferSize) + notFull.await(); + + if (error != null) { + throw error; + } + + AsyncCollector collector = new AsyncCollector(this, true); + + collectorToQueue.put(collector, queue.add(collector)); + collectorToStreamElement.put(collector, mark); + + // 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 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() { + emitter.stop(); + + emitThread.interrupt(); + } + + /** + * Get all StreamElements in the AsyncCollector queue. + *

+ * Emitter Thread can not output records and will wait for a while due to isCheckpointing flag + * until doing checkpoint has done. + * + * @return A List containing StreamElements. + */ + public List getStreamElementsInBuffer() { + try { + lock.lock(); + + // stop emitter thread + isCheckpointing = true; + + List ret = new ArrayList<>(); + for (int i = 0; i < queue.size(); ++i) { + AsyncCollector collector = queue.get(i); + ret.add(collectorToStreamElement.get(collector)); + } + + return ret; + } + finally { + lock.unlock(); + } + } + + public void setOutput(TimestampedCollector collector, Output> output) { + this.timestampedCollector = collector; + this.output = output; + } + + public void notifyCheckpointDone() { + this.isCheckpointing = false; + this.taskDone.signalAll(); + } + + /** + * A working thread to output results from {@link AsyncCollector} to the next operator. + */ + private class Emitter implements Runnable { + private volatile boolean running = true; + + private void output(AsyncCollector collector) throws Exception { + List result = collector.getResult(); + + // update timestamp for output stream records based on the input stream record. + StreamElement element = collectorToStreamElement.get(collector); + if (element == null) { + throw new Exception("No input stream record or watermark for current AsyncCollector: "+collector); + } + + if (element.isRecord()) { + if (result == null) { + throw new Exception("Result for stream record "+element+" is null"); + } + + timestampedCollector.setTimestamp(element.asRecord()); + for (OUT val : result) { + timestampedCollector.collect(val); + } + } + else if (element.isWatermark()) { + output.emitWatermark(element.asWatermark()); + } + else if (element.isLatencyMarker()) { + operator.sendLatencyMarker(element.asLatencyMarker()); + } + else { + throw new Exception("Unknown input record: "+element); + } + } + + private void clearInfoInMaps(AsyncCollector collector) { + collectorToStreamElement.remove(collector); + collectorToQueue.remove(collector); + } + + /** + * Emit results from the finished head collector and its following finished ones. + */ + private void orderedProcess() { + while (queue.size() > 0) { + try { + AsyncCollector collector = queue.get(0); + if (!collector.isDone()) { + break; + } + + output(collector); + + queue.remove(0); + clearInfoInMaps(collector); + + notFull.signal(); + } + catch (Exception e) { + error = e; + break; + } + } + } + + /** + * Emit results for each finished collector. + */ + private void unorderedProcess() { + AsyncCollector collector = finishedCollectors.pollFirst(); + while (collector != null) { + try { + output(collector); + + queue.remove(collectorToQueue.get(collector)); + clearInfoInMaps(collector); + + notFull.signal(); + + collector = finishedCollectors.pollFirst(); + } + catch (Exception e) { + error = e; + break; + } + } + } + + /** + * If some bad things happened(like exceptions from async i/o), the operator tries to fail + * itself at: + * {@link AsyncWaitOperator#processElement}, triggered by calling {@link AsyncCollectorBuffer#add}. + * {@link AsyncWaitOperator#snapshotState} + * {@link AsyncWaitOperator#close} while calling {@link AsyncCollectorBuffer#waitEmpty} + * + * It is necessary for Emitter Thread to notify methods blocking on notFull/isEmpty. + */ + private void processError() { + queue.clear(); + finishedCollectors.clear(); + collectorToQueue.clear(); + collectorToStreamElement.clear(); + + notFull.signalAll(); + isEmpty.signalAll(); + } + + /** + * If + * In ordered mode, there are some finished async collectors, and one of them is the first element in + * the queue. + * or + * In unordered mode, there are some finished async collectors. + * or + * The first element in the queue is Watermark. + * Then, the emitter thread should keep waiting, rather than waiting on the condition. + * + * Otherwise, the thread should stop for a while until being signalled. + */ + private boolean nothingToDo() { + if (queue.size() == 0) { + isEmpty.signalAll(); + return true; + } + + // while doing checkpoints, emitter thread should not try to output records. + if (isCheckpointing) { + return true; + } + + // check head element of the queue, it is OK to process Watermark or LatencyMarker + if (collectorToStreamElement.get(queue.get(0)).isWatermark() || + collectorToStreamElement.get(queue.get(0)).isLatencyMarker()) + { + return false; + } + + if (mode == AsyncDataStream.OutputMode.UNORDERED) { + // no finished async collector... + if (finishedCollectors.size() == 0) { + return true; + } + else { + return false; + } + } + else { + // for ORDERED mode, make sure the first collector in the queue has been done. + AsyncCollector collector = queue.get(0); + if (collector.isDone() == false) { + return true; + } + else { + return false; + } + } + } + + @Override + public void run() { + while (running) { + try { + lock.lock(); + + if (error != null) { + // stop processing finished async collectors, and try to wake up blocked main + // thread or checkpoint thread repeatedly. + processError(); --- End diff -- Is the `Emitter` supposed to continue working after an error has occurred? If not, then we should probably stop the thread by leaving the while loop. --- 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 infrastructure@apache.org or file a JIRA ticket with INFRA. ---