Thanks for your reply, Ahmed. > (2) The FLIP-451 aims to introduce a timeout configuration, but I didn’t >> find the configuration in FLIP even I lookup some historical versions of >> the FLIP. Did I miss some key informations? >> > > Yes, I tried to implicitly point that it will be added to the existing > AsyncSinkWriterConfiguration to not inflate the FLIP, but I get it might be > confusing. I have added the changes to the configuration classes in the > FLIP to make it clearer.
(1) Implicitly point a public API change is not enough, Could you add a section Public Interfaces to enumerate all Public APIs that you proposed and you changed? It’s a standard part of a FLIP template[1]. (2) About the proposed public interface ResultHandler<RequestEntryT>, Could you explain or show how to use the methods #completeExceptionally and #retryForEntries? I didn’t find detail explanation or Usage example code to understand them. (3) Could you add necessary java documents for all public API changes like new method AsyncSinkWriterConfiguration#setRequestTimeoutMs ? The java doc of [2] is a good example. (4) Another minor reminder AsyncSinkBase is a @PublicEvolving interface too, please correct it, and please ensure the backward compatibility has been considered for all public interfaces the FLIP changed. Best, Leonard [1]https://cwiki.apache.org/confluence/display/FLINK/FLIP+Template [2]https://cwiki.apache.org/confluence/display/FLINK/FLIP-171%3A+Async+Sink > > > On Tue, 21 May 2024 at 14:56, Leonard Xu <xbjt...@gmail.com> wrote: > >> Thanks Ahmed for kicking off this discussion, sorry for jumping the >> discussion late. >> >> (1)I’m confused about the discuss thread name ‘FLIP-451: Refactor Async >> sink API’ and FLIP title/vote thread name ' >> FLIP-451: Introduce timeout configuration to AsyncSink API < >> https://cwiki.apache.org/confluence/display/FLINK/FLIP-451%3A+Introduce+timeout+configuration+to+AsyncSink+API>’, >> they are different for me. Could you help explain the change history? >> >> (2) The FLIP-451 aims to introduce a timeout configuration, but I didn’t >> find the configuration in FLIP even I lookup some historical versions of >> the FLIP. Did I miss some key informations? >> >> (3) About the code change part, there’re some un-complete pieces in >> AsyncSinkWriter for example `submitRequestEntries(List<RequestEntryT> >> requestEntries,);` is incorrect and `sendTime` variable I didn’t >> find the place we define it and where we use it. >> >> Sorry for jumping the discussion thread during vote phase again. >> >> Best, >> Leonard >> >> >>> 2024年5月21日 下午3:49,Ahmed Hamdy <hamdy10...@gmail.com> 写道: >>> >>> Hi Hong, >>> Thanks for pointing that out, no we are not >>> deprecating getFatalExceptionCons(). I have updated the FLIP >>> Best Regards >>> Ahmed Hamdy >>> >>> >>> On Mon, 20 May 2024 at 15:40, Hong Liang <h...@apache.org> wrote: >>> >>>> Hi Ahmed, >>>> Thanks for putting this together! Should we still be marking >>>> getFatalExceptionCons() as @Deprecated in this FLIP, if we are not >>>> providing a replacement? >>>> >>>> Regards, >>>> Hong >>>> >>>> On Mon, May 13, 2024 at 7:58 PM Ahmed Hamdy <hamdy10...@gmail.com> >> wrote: >>>> >>>>> Hi David, >>>>> yes there error classification was initially left to sink implementers >> to >>>>> handle while we provided utilities to classify[1] and bubble up[2] >> fatal >>>>> exceptions to avoid retrying them. >>>>> Additionally some sink implementations provide an option to short >> circuit >>>>> the failures by exposing a `failOnError` flag as in >>>> KinesisStreamsSink[3], >>>>> however this FLIP scope doesn't include any changes for retry >> mechanisms. >>>>> >>>>> 1- >>>>> >>>>> >>>> >> https://github.com/apache/flink/blob/015867803ff0c128b1c67064c41f37ca0731ed86/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/throwable/FatalExceptionClassifier.java#L32 >>>>> 2- >>>>> >>>>> >>>> >> https://github.com/apache/flink/blob/015867803ff0c128b1c67064c41f37ca0731ed86/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java#L533 >>>>> 3- >>>>> >>>>> >>>> >> https://github.com/apache/flink-connector-aws/blob/c6e0abb65a0e51b40dd218b890a111886fbf797f/flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisStreamsSinkWriter.java#L106 >>>>> >>>>> Best Regards >>>>> Ahmed Hamdy >>>>> >>>>> >>>>> On Mon, 13 May 2024 at 16:20, David Radley <david_rad...@uk.ibm.com> >>>>> wrote: >>>>> >>>>>> Hi, >>>>>> I wonder if the way that the async request fails could be a retriable >>>> or >>>>>> non-retriable error, so it would retry only for retriable (transient) >>>>>> errors (like IOExceptions) . I see some talk on the internet around >>>>>> retriable SQL errors. >>>>>> If this was the case then we may need configuration to limit the >>>> number >>>>>> of retries of retriable errors. >>>>>> Kind regards, David >>>>>> >>>>>> >>>>>> From: Muhammet Orazov <mor+fl...@morazow.com.INVALID> >>>>>> Date: Monday, 13 May 2024 at 10:30 >>>>>> To: dev@flink.apache.org <dev@flink.apache.org> >>>>>> Subject: [EXTERNAL] Re: [DISCUSS] FLIP-451: Refactor Async sink API >>>>>> Great, thanks for clarifying! >>>>>> >>>>>> Best, >>>>>> Muhammet >>>>>> >>>>>> >>>>>> On 2024-05-06 13:40, Ahmed Hamdy wrote: >>>>>>> Hi Muhammet, >>>>>>> Thanks for the feedback. >>>>>>> >>>>>>>> Could you please add more here why it is harder? Would the >>>>>>>> `completeExceptionally` >>>>>>>> method be related to it? Maybe you can add usage example for it >>>> also. >>>>>>>> >>>>>>> >>>>>>> this is mainly due to the current implementation of fatal exception >>>>>>> failures which depends on base `getFatalExceptionConsumer` method >>>> that >>>>>>> is >>>>>>> decoupled from the actual called method `submitRequestEntries`, Since >>>>>>> this >>>>>>> is now not the primary concern of the FLIP, I have removed it from >>>> the >>>>>>> motivation so that the scope is defined around introducing the >>>> timeout >>>>>>> configuration. >>>>>>> >>>>>>>> Should we add a list of possible connectors that this FLIP would >>>>>>>> improve? >>>>>>> >>>>>>> Good call, I have added under migration plan. >>>>>>> >>>>>>> Best Regards >>>>>>> Ahmed Hamdy >>>>>>> >>>>>>> >>>>>>> On Mon, 6 May 2024 at 08:49, Muhammet Orazov <mor+fl...@morazow.com> >>>>>>> wrote: >>>>>>> >>>>>>>> Hey Ahmed, >>>>>>>> >>>>>>>> Thanks for the FLIP! +1 (non-binding) >>>>>>>> >>>>>>>>> Additionally the current interface for passing fatal exceptions >>>> and >>>>>>>>> retrying records relies on java consumers which makes it harder to >>>>>>>>> understand. >>>>>>>> >>>>>>>> Could you please add more here why it is harder? Would the >>>>>>>> `completeExceptionally` >>>>>>>> method be related to it? Maybe you can add usage example for it >>>> also. >>>>>>>> >>>>>>>>> we should proceed by adding support in all supporting connector >>>>> repos. >>>>>>>> >>>>>>>> Should we add list of possible connectors that this FLIP would >>>>>>>> improve? >>>>>>>> >>>>>>>> Best, >>>>>>>> Muhammet >>>>>>>> >>>>>>>> >>>>>>>> On 2024-04-29 14:08, Ahmed Hamdy wrote: >>>>>>>>> Hi all, >>>>>>>>> I would like to start a discussion on FLIP-451[1] >>>>>>>>> The proposal comes on encountering a couple of issues while >>>> working >>>>>>>>> with >>>>>>>>> implementers for Async Sink. >>>>>>>>> The FLIP mainly proposes a new API similar to AsyncFunction and >>>>>>>>> ResultFuture as well as introducing timeout handling for AsyncSink >>>>>>>>> requests. >>>>>>>>> The FLIP targets 1.20 with backward compatible changes and we >>>> should >>>>>>>>> proceed by adding support in all supporting connector repos. >>>>>>>>> >>>>>>>>> 1- >>>>>>>>> >>>>>>>> >>>>>> >>>>> >>>> >> https://cwiki.apache.org/confluence/display/FLINK/FLIP-451%3A+Refactor+Async+Sink+API >>>>>>>>> Best Regards >>>>>>>>> Ahmed Hamdy >>>>>>>> >>>>>> >>>>>> Unless otherwise stated above: >>>>>> >>>>>> IBM United Kingdom Limited >>>>>> Registered in England and Wales with number 741598 >>>>>> Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 >> 3AU >>>>>> >>>>> >>>> >> >>