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 78928200BCA for ; Mon, 21 Nov 2016 12:32:10 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 77D28160AEC; Mon, 21 Nov 2016 11:32:10 +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 6A64C160AF9 for ; Mon, 21 Nov 2016 12:32:09 +0100 (CET) Received: (qmail 8071 invoked by uid 500); 21 Nov 2016 11:32:08 -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 8006 invoked by uid 99); 21 Nov 2016 11:32:08 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 21 Nov 2016 11:32:08 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 230372C4C70 for ; Mon, 21 Nov 2016 11:32:08 +0000 (UTC) Date: Mon, 21 Nov 2016 11:32:08 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-4391) Provide support for asynchronous operations over streams MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 21 Nov 2016 11:32:10 -0000 [ https://issues.apache.org/jira/browse/FLINK-4391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15683287#comment-15683287 ] ASF GitHub Bot commented on FLINK-4391: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2629#discussion_r88701684 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncCollectorBuffer.java --- @@ -0,0 +1,453 @@ +/* + * 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.annotation.VisibleForTesting; +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 java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * 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 { + + /** + * Max number of {@link AsyncCollector} in the buffer. + */ + private final int bufferSize; + + private final AsyncDataStream.OutputMode mode; + + private final AsyncWaitOperator operator; + + /** + * Keep all {@code AsyncCollector} and their input {@link StreamElement} + */ + private final Map, StreamElement> queue = new LinkedHashMap<>(); + /** + * For the AsyncWaitOperator chained with StreamSource, the checkpoint thread may get the + * {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock} while {@link AsyncCollectorBuffer#queue} + * is full since main thread waits on this lock. The StreamElement in + * {@link AsyncWaitOperator#processElement(StreamRecord)} should be treated as a part of all StreamElements + * in its queue. It will be kept in the operator state while snapshotting. + */ + private StreamElement extraStreamElement; + + /** + * {@link TimestampedCollector} and {@link Output} to collect results and watermarks. + */ + private final Output> output; + private final TimestampedCollector timestampedCollector; + + /** + * Checkpoint lock from {@link org.apache.flink.streaming.runtime.tasks.StreamTask#lock} + */ + private final Object lock; + + private final Emitter emitter; + private final Thread emitThread; + + private IOException error; + + public AsyncCollectorBuffer( + int bufferSize, + AsyncDataStream.OutputMode mode, + Output> output, + TimestampedCollector collector, + Object lock, + AsyncWaitOperator operator) { + Preconditions.checkArgument(bufferSize > 0, "Future buffer size should be greater than 0."); + Preconditions.checkNotNull(output, "Output should not be NULL."); + Preconditions.checkNotNull(collector, "TimestampedCollector should not be NULL."); + Preconditions.checkNotNull(lock, "Checkpoint lock should not be NULL."); + Preconditions.checkNotNull(operator, "Reference to AsyncWaitOperator should not be NULL."); + + this.bufferSize = bufferSize; + this.mode = mode; + this.output = output; + this.timestampedCollector = collector; + this.operator = operator; + this.lock = lock; + + 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 a new space is available. + * + * @param record StreamRecord + * @return An AsyncCollector + * @throws Exception InterruptedException or IOException from AsyncCollector. + */ + public AsyncCollector addStreamRecord(StreamRecord record) throws InterruptedException, IOException { + while (queue.size() >= bufferSize) { + // hold the input StreamRecord until it is placed in the buffer + extraStreamElement = record; + + lock.wait(); + } + + if (error != null) { + throw error; + } + + AsyncCollector collector = new AsyncCollector(this); + + queue.put(collector, record); + + extraStreamElement = null; + + return collector; + } + + /** + * Add a {@link Watermark} into queue. A new AsyncCollector will be created and returned. + *

+ * If queue is full, caller will wait here. + * + * @param watermark Watermark + * @return AsyncCollector + * @throws Exception InterruptedException or IOException from AsyncCollector. --- End diff -- Does not throw an `Exception`. > Provide support for asynchronous operations over streams > -------------------------------------------------------- > > Key: FLINK-4391 > URL: https://issues.apache.org/jira/browse/FLINK-4391 > Project: Flink > Issue Type: New Feature > Components: DataStream API > Reporter: Jamie Grier > Assignee: david.wang > > Many Flink users need to do asynchronous processing driven by data from a DataStream. The classic example would be joining against an external database in order to enrich a stream with extra information. > It would be nice to add general support for this type of operation in the Flink API. Ideally this could simply take the form of a new operator that manages async operations, keeps so many of them in flight, and then emits results to downstream operators as the async operations complete. -- This message was sent by Atlassian JIRA (v6.3.4#6332)