[jira] [Created] (FLINK-32895) Introduce the max attempts for Exponential Delay Restart Strategy

2023-08-18 Thread Rui Fan (Jira)
Rui Fan created FLINK-32895: --- Summary: Introduce the max attempts for Exponential Delay Restart Strategy Key: FLINK-32895 URL: https://issues.apache.org/jira/browse/FLINK-32895 Project: Flink

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-18 Thread Becket Qin
Thanks for the proposal, Venkata. The current proposal works if we do not want to migrate SupportsFilterPushdown to also use NestedFieldReferenceExpression in the long term. Otherwise, the alternative solution briefly mentioned in the rejected alternatives would be the following: Phase 1: 1.

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Galen Warren
Gotcha, makes sense as to the original division. >> Can this be solved by simply passing in the path to the artifacts This definitely works if we're going to be copying the artifacts on the host side -- into the build context -- and then from the context into the image. It only gets tricky to

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Tzu-Li (Gordon) Tai
Hi Galen, The original intent of having a separate repo for the playground repo, was that StateFun users can just go to that and start running simple examples without any other distractions from the core code. I personally don't have a strong preference here and can understand how it would make

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Galen Warren
Yes, exactly! And in addition to the base Statefun jars and the jar for the Java SDK, it does an equivalent copy/register operation for each of the other SDK libraries (Go, Python, Javascript) so that those libraries are also available when building the playground examples. One more question: In

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Tzu-Li (Gordon) Tai
Hi Galen, > locally built code is copied into the build containers so that it can be accessed during the build. That's exactly what we had been doing for release testing, yes. Sorry I missed that detail in my previous response. And yes, that sounds like a reasonable approach. If I understand

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Galen Warren
Thanks. If you were to build a local image, as you suggest, how do you access that image when building the playground images? All the compilation of playground code happens inside containers, so local images on the host aren't available in those containers. Unless I'm missing something? I've

Re: [DISCUSS] Status of Statefun Project

2023-08-18 Thread Tzu-Li (Gordon) Tai
Hi Galen, > Gordon, is there a trick to running the sample code in flink-statefun-playground against yet-unreleased code that I'm missing? You'd have to locally build an image from the release branch, with a temporary image version tag. Then, in the flink-statefun-playground, change the image

[DISCUSS] FLIP-319: Integrating with Kafka’s proper support for 2PC participation (KIP-939).

2023-08-18 Thread Tzu-Li (Gordon) Tai
Hi Flink devs, I’d like to officially start a discussion for FLIP-319: Integrating with Kafka’s proper support for 2PC participation (KIP-939) [1]. This is the “sister” joint FLIP for KIP-939 [2] [3]. It has been a long-standing issue that Flink’s Kafka connector doesn’t work fully correctly

Re: [DISCUSS] FLIP-356: Support Nested Fields Filter Pushdown

2023-08-18 Thread Venkatakrishnan Sowrirajan
Gentle ping for reviews/feedback. On Tue, Aug 15, 2023, 5:37 PM Venkatakrishnan Sowrirajan wrote: > Hi All, > > I am opening this thread to discuss FLIP-356: Support Nested Fields > Filter Pushdown. The FLIP can be found at >

Re: FLINK-20767 - Support for nested fields filter push down

2023-08-18 Thread Venkatakrishnan Sowrirajan
Gentle ping On Wed, Aug 16, 2023, 11:56 AM Venkatakrishnan Sowrirajan wrote: > Forgot to share the link - > https://lists.apache.org/thread/686bhgwrrb4xmbfzlk60szwxos4z64t7 in the > last email. > > Regards > Venkata krishnan > > > On Wed, Aug 16, 2023 at 11:55 AM Venkatakrishnan Sowrirajan < >

Re: [DISCUSS] [FLINK-32873] Add a config to allow disabling Query hints

2023-08-18 Thread Timo Walther
> lots of the streaming SQL syntax are extensions of SQL standard That is true. But hints are kind of a special case because they are not even "part of Flink SQL" that's why they are written in a comment syntax. Anyway, I feel hints could be sometimes confusing for users because most of them

Re: [DISCUSS] FLIP-348: Support System Columns in SQL and Table API

2023-08-18 Thread Timo Walther
Great, I also like my last suggestion as it is even more elegant. I will update the FLIP until Monday. Regards, Timo On 17.08.23 13:55, Jark Wu wrote: Hi Timo, I'm fine with your latest suggestion that introducing a flag to control expanding behavior of metadata virtual columns, but not

Re: [DISCUSS] FLIP-327: Support stream-batch unified operator to improve job throughput when processing backlog data

2023-08-18 Thread Dong Lin
Hi Piotr, Thanks for the explanation. To recap our offline discussion, there is a concern regarding the capability to dynamically switch between stream and batch modes. This concern is around unforeseen behaviors such as bugs or performance regressions, which we might not yet be aware of yet.

[jira] [Created] (FLINK-32894) flink-connector-parent should use maven-shade-plugin 3.3.0+ for Java 17

2023-08-18 Thread Qingsheng Ren (Jira)
Qingsheng Ren created FLINK-32894: - Summary: flink-connector-parent should use maven-shade-plugin 3.3.0+ for Java 17 Key: FLINK-32894 URL: https://issues.apache.org/jira/browse/FLINK-32894 Project: