You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by kisimple <gi...@git.apache.org> on 2018/05/10 08:55:47 UTC

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

GitHub user kisimple opened a pull request:

    https://github.com/apache/flink/pull/5983

    [FLINK-7789][DataStream API] Add handler for Async IO operator timeouts

    ## What is the purpose of the change
    
    *Currently Async IO operator does not provide a mechanism to handle timeouts. This PR fixs the problem by adding an enhanced `AsyncFunction`, named `TimeoutAwareAsyncFunction`.*
    
    
    ## Brief change log
    
      - *Add a new interface, `TimeoutAwareAsyncFunction`, which extends the `AsyncFunction`*
      - *Change `RichAsyncFunction` to implement `TimeoutAwareAsyncFunction` instead of `AsyncFunction`*
      - *`AsyncWaitOperator` will invoke `TimeoutAwareAsyncFunction#timeout` when `asyncInvoke` times out*
    
    
    ## Verifying this change
    
      - *Add tests to `AsyncWaitOperatorTest`*
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): ( no )
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: ( yes )
      - The serializers: ( no )
      - The runtime per-record code paths (performance sensitive): ( yes )
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: ( no )
      - The S3 file system connector: ( no )
    
    ## Documentation
    
      - Does this pull request introduce a new feature? ( yes )
      - If yes, how is the feature documented? ( docs / JavaDocs )


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kisimple/flink FLINK-7789

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5983.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5983
    
----
commit 7cfb7ee8c7d29f7f4e004680ff1debcfcd897bff
Author: blueszheng <ki...@...>
Date:   2018-05-10T06:41:34Z

    [FLINK-7789] Add handler for Async IO operator timeouts

----


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5983#discussion_r189244151
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/TimeoutAwareAsyncFunction.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.annotation.PublicEvolving;
    +
    +/**
    + * An enhanced {@link AsyncFunction} which can handle timeouts.
    + */
    +@PublicEvolving
    +public interface TimeoutAwareAsyncFunction<IN, OUT> extends AsyncFunction<IN, OUT> {
    +
    +	/**
    +	 * asyncInvoke timeout occurred.
    +	 * Here you can complete the result future exceptionally with timeout exception,
    +	 * or complete with empty result. You can also retry to complete with the right results.
    +	 *
    +	 * @param input element coming from an upstream task
    +	 * @param resultFuture to be completed with the result data
    +	 * @exception Exception in case of a user code error. An exception will make the task fail and
    +	 * trigger fail-over process.
    +	 */
    +	void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception;
    --- End diff --
    
    Wouldn't it be enough to add such `timeout()` method to  `AsyncFunction` with default implementation that fails the `ResultFuture`? I mean instead of adding new interface and deprecating `AsyncFunction`?


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5983#discussion_r190567872
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/TimeoutAwareAsyncFunction.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.annotation.PublicEvolving;
    +
    +/**
    + * An enhanced {@link AsyncFunction} which can handle timeouts.
    + */
    +@PublicEvolving
    +public interface TimeoutAwareAsyncFunction<IN, OUT> extends AsyncFunction<IN, OUT> {
    +
    +	/**
    +	 * asyncInvoke timeout occurred.
    +	 * Here you can complete the result future exceptionally with timeout exception,
    +	 * or complete with empty result. You can also retry to complete with the right results.
    +	 *
    +	 * @param input element coming from an upstream task
    +	 * @param resultFuture to be completed with the result data
    +	 * @exception Exception in case of a user code error. An exception will make the task fail and
    +	 * trigger fail-over process.
    +	 */
    +	void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception;
    --- End diff --
    
    No problem :) Please CC me if you open next RP for this issue.


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by kisimple <gi...@git.apache.org>.
Github user kisimple commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5983#discussion_r190560531
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/TimeoutAwareAsyncFunction.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.annotation.PublicEvolving;
    +
    +/**
    + * An enhanced {@link AsyncFunction} which can handle timeouts.
    + */
    +@PublicEvolving
    +public interface TimeoutAwareAsyncFunction<IN, OUT> extends AsyncFunction<IN, OUT> {
    +
    +	/**
    +	 * asyncInvoke timeout occurred.
    +	 * Here you can complete the result future exceptionally with timeout exception,
    +	 * or complete with empty result. You can also retry to complete with the right results.
    +	 *
    +	 * @param input element coming from an upstream task
    +	 * @param resultFuture to be completed with the result data
    +	 * @exception Exception in case of a user code error. An exception will make the task fail and
    +	 * trigger fail-over process.
    +	 */
    +	void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception;
    --- End diff --
    
    Sorry for that I did not pay so much attention to new features. I will try and if it works I will close this PR and open a new one. Thanks:)


---

[GitHub] flink issue #5983: [FLINK-7789][DataStream API] Add handler for Async IO ope...

Posted by kisimple <gi...@git.apache.org>.
Github user kisimple commented on the issue:

    https://github.com/apache/flink/pull/5983
  
    cc @tillrohrmann @kl0u 


---

[GitHub] flink issue #5983: [FLINK-7789][DataStream API] Add handler for Async IO ope...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/5983
  
    This PR superseded by https://github.com/apache/flink/pull/6091


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by kisimple <gi...@git.apache.org>.
Github user kisimple closed the pull request at:

    https://github.com/apache/flink/pull/5983


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5983#discussion_r190553816
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/TimeoutAwareAsyncFunction.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.annotation.PublicEvolving;
    +
    +/**
    + * An enhanced {@link AsyncFunction} which can handle timeouts.
    + */
    +@PublicEvolving
    +public interface TimeoutAwareAsyncFunction<IN, OUT> extends AsyncFunction<IN, OUT> {
    +
    +	/**
    +	 * asyncInvoke timeout occurred.
    +	 * Here you can complete the result future exceptionally with timeout exception,
    +	 * or complete with empty result. You can also retry to complete with the right results.
    +	 *
    +	 * @param input element coming from an upstream task
    +	 * @param resultFuture to be completed with the result data
    +	 * @exception Exception in case of a user code error. An exception will make the task fail and
    +	 * trigger fail-over process.
    +	 */
    +	void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception;
    --- End diff --
    
    Can not we add default method here (Java 8 feature)?


---

[GitHub] flink pull request #5983: [FLINK-7789][DataStream API] Add handler for Async...

Posted by kisimple <gi...@git.apache.org>.
Github user kisimple commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5983#discussion_r189250250
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/TimeoutAwareAsyncFunction.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.annotation.PublicEvolving;
    +
    +/**
    + * An enhanced {@link AsyncFunction} which can handle timeouts.
    + */
    +@PublicEvolving
    +public interface TimeoutAwareAsyncFunction<IN, OUT> extends AsyncFunction<IN, OUT> {
    +
    +	/**
    +	 * asyncInvoke timeout occurred.
    +	 * Here you can complete the result future exceptionally with timeout exception,
    +	 * or complete with empty result. You can also retry to complete with the right results.
    +	 *
    +	 * @param input element coming from an upstream task
    +	 * @param resultFuture to be completed with the result data
    +	 * @exception Exception in case of a user code error. An exception will make the task fail and
    +	 * trigger fail-over process.
    +	 */
    +	void timeout(IN input, ResultFuture<OUT> resultFuture) throws Exception;
    --- End diff --
    
    Thanks for your review. The deprecated `AsyncFunction` is a Java Interface which can not have a method body due to Java grammars. However, your comment make me realize that I just forgot about the Scala API for `AsyncFunction`, so there is more work need to be done.


---