flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4391) Provide support for asynchronous operations over streams
Date Mon, 21 Nov 2016 11:32:05 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15683277#comment-15683277
] 

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_r88680106
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/AsyncFunction.java
---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.functions.async;
    +
    +import org.apache.flink.api.common.functions.Function;
    +import org.apache.flink.streaming.api.operators.async.AsyncCollector;
    +
    +import java.io.Serializable;
    +
    +/**
    + * A function to trigger Async I/O operation.
    + * <p>
    + * For each #asyncInvoke, an async io operation can be triggered, and once it has been
done,
    + * the result can be collected by calling {@link AsyncCollector#collect}. For each async
    + * operations, their contexts are buffered in the operator immediately after invoking
    + * #asyncInvoke, leading to no blocking for each stream input as long as internal buffer
is not full.
    + * <p>
    + * {@link AsyncCollector} can be passed into callbacks or futures provided by async client
to
    + * fetch result data. Any error can also be propagate to the operator by {@link AsyncCollector#collect(Throwable)}.
    + *
    + * <p>
    + * Typical usage for callback:
    + * <pre>{@code
    + * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
    + *   @Override
    + *   public void asyncInvoke(String row, AsyncCollector<String> collector) throws
Exception {
    + *     HBaseCallback cb = new HBaseCallback(collector);
    + *     Get get = new Get(Bytes.toBytes(row));
    + *     hbase.asyncGet(get, cb);
    + *   }
    + * }
    + * }
    + * </pre>
    + *
    + * <p>
    + * Typical usage for {@link com.google.common.util.concurrent.ListenableFuture}
    + * <pre>{@code
    + * public class HBaseAsyncFunc implements AsyncFunction<String, String> {
    + *   @Override
    + *   public void asyncInvoke(String row, final AsyncCollector<String> collector)
throws Exception {
    + *     Get get = new Get(Bytes.toBytes(row));
    + *     ListenableFuture<Result> future = hbase.asyncGet(get);
    + *     Futures.addCallback(future, new FutureCallback<Result>() {
    + *       public void onSuccess(Result result) {
    + *         List<String> ret = process(result);
    + *         collector.collect(ret);
    + *       }
    + *       public void onFailure(Throwable thrown) {
    + *         collector.collect(thrown);
    + *       }
    + *     });
    + *   }
    + * }
    + * }
    + * </pre>
    + *
    + * @param <IN> The type of the input elements.
    + * @param <OUT> The type of the returned elements.
    + */
    +public interface AsyncFunction<IN, OUT> extends Function, Serializable {
    --- End diff --
    
    `PublicEvolving`


> 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)

Mime
View raw message