Re: [VOTE] Deprecate Per-Job Mode in Flink 1.15

2022-02-02 Thread Xintong Song
Thanks for the clarification, Konstantin. +1 for deprecating per-job mode in Flink 1.15, and reevaluating when to drop it after Flink 1.16. Thank you~ Xintong Song On Tue, Feb 1, 2022 at 5:27 PM Konstantin Knauf wrote: > Hi Xintong, Hi Yang, Hi everyone, > > Thank you for speaking up. The v

[jira] [Created] (FLINK-25936) MutableTypeCell is not setting correctly the value's type_name

2022-02-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-25936: Summary: MutableTypeCell is not setting correctly the value's type_name Key: FLINK-25936 URL: https://issues.apache.org/jira/browse/FLINK-25936 Project: Flink

Re: Statefun async http request via RequestReplyFunctionBuilder

2022-02-02 Thread Galen Warren
Ah, nevermind. I was misunderstanding how maxRequestDuration related to the retries. Locally, I had Flink set not to retry on failure, so once maxRequestDuration expired without a successful result, Flink itself was stopping, and I wasn't seeing the function get called again. But I see now that, i

Re: Statefun async http request via RequestReplyFunctionBuilder

2022-02-02 Thread Igal Shilman
Great, ping me when you would like to pick this up. For the related issue, I think that can be a good addition indeed! On Wed, Feb 2, 2022 at 8:55 PM Galen Warren wrote: > Gotcha, thanks. I may be able to work on that one in a couple weeks if > you're looking for help. > > Unrelated question --

[jira] [Created] (FLINK-25935) Add a harness based entry point to simply getting started.

2022-02-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-25935: Summary: Add a harness based entry point to simply getting started. Key: FLINK-25935 URL: https://issues.apache.org/jira/browse/FLINK-25935 Project: Flink Is

[jira] [Created] (FLINK-25934) Modernize statefun playground examples

2022-02-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-25934: Summary: Modernize statefun playground examples Key: FLINK-25934 URL: https://issues.apache.org/jira/browse/FLINK-25934 Project: Flink Issue Type: Improvemen

Re: Statefun async http request via RequestReplyFunctionBuilder

2022-02-02 Thread Galen Warren
Gotcha, thanks. I may be able to work on that one in a couple weeks if you're looking for help. Unrelated question -- another thing that would be useful for me would be the ability to set a maximum backoff interval in BoundedExponentialBackoff or the async equivalent. My situation is this. I'd lik

Re: Statefun async http request via RequestReplyFunctionBuilder

2022-02-02 Thread Igal Shilman
Hi Galen, You are right, it is not possible, but there is no real reason for that. We should fix this, and I've created the following JIRA issue [1] [1] https://issues.apache.org/jira/browse/FLINK-25933 On Wed, Feb 2, 2022 at 6:30 PM Galen Warren wrote: > Is it possible to choose the async HTT

[jira] [Created] (FLINK-25933) Allow configuring different transports in RequestReplyFunctionBuilder

2022-02-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-25933: Summary: Allow configuring different transports in RequestReplyFunctionBuilder Key: FLINK-25933 URL: https://issues.apache.org/jira/browse/FLINK-25933 Project: Flink

Re: Need help with finding inner workings of watermark stream idleness

2022-02-02 Thread Jeff Carter
Thanks, Seth. Yea this looks perfect. I had a feeling I'd need to get deep into things, and no time like the present haha. May ask for more guidance with those inner workings to get a bit of a road map. But that gets into the feature idea and beyond the scope of this thread's original question so

Statefun async http request via RequestReplyFunctionBuilder

2022-02-02 Thread Galen Warren
Is it possible to choose the async HTTP transport using RequestReplyFunctionBuilder? It looks to me that it is not, but I wanted to double check. Thanks.

[jira] [Created] (FLINK-25932) Introduce ExecNodeContext.generateUid()

2022-02-02 Thread Timo Walther (Jira)
Timo Walther created FLINK-25932: Summary: Introduce ExecNodeContext.generateUid() Key: FLINK-25932 URL: https://issues.apache.org/jira/browse/FLINK-25932 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-25931) Add projection pushdown support for CsvFormatFactory

2022-02-02 Thread Alexander Fedulov (Jira)
Alexander Fedulov created FLINK-25931: - Summary: Add projection pushdown support for CsvFormatFactory Key: FLINK-25931 URL: https://issues.apache.org/jira/browse/FLINK-25931 Project: Flink

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-02 Thread Gyula Fóra
Hi Danny! Thanks for the feedback :) Versioning: Versioning will be independent from Flink and the operator will depend on a fixed flink version (in every given operator version). This should be the exact same setup as with Stateful Functions ( https://github.com/apache/flink-statefun). So indepe

Re: [DISCUSS] FLIP-212: Introduce Flink Kubernetes Operator

2022-02-02 Thread Danny Cranmer
Hey team, Great work on the FLIP, I am looking forward to this one. I agree that we can move forward to the voting stage. I have general feedback around how we will handle job submission failure and retry. As discussed in the Rejected Alternatives section, we can use Java to handle job submission

Re: Off for a week starting Friday

2022-02-02 Thread Till Rohrmann
Thanks for letting us know Etienne. Have a nice time off :-) Cheers, Till On Wed, Feb 2, 2022 at 3:56 PM Etienne Chauchot wrote: > Hi all, > > I'll be off for a week starting Friday afternoon so I might be > unresponsive on ongoing PRs/tickets. > > Best > > Etienne. > >

Off for a week starting Friday

2022-02-02 Thread Etienne Chauchot
Hi all, I'll be off for a week starting Friday afternoon so I might be unresponsive on ongoing PRs/tickets. Best Etienne.

[jira] [Created] (FLINK-25930) Remove identity casting from ScalarOperatorGens

2022-02-02 Thread Marios Trivyzas (Jira)
Marios Trivyzas created FLINK-25930: --- Summary: Remove identity casting from ScalarOperatorGens Key: FLINK-25930 URL: https://issues.apache.org/jira/browse/FLINK-25930 Project: Flink Issue T

[jira] [Created] (FLINK-25929) Jepsen tests don't work with Zookeeper 3.4

2022-02-02 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-25929: Summary: Jepsen tests don't work with Zookeeper 3.4 Key: FLINK-25929 URL: https://issues.apache.org/jira/browse/FLINK-25929 Project: Flink Issue Type

[jira] [Created] (FLINK-25928) Refactor timestamp<->number validation messages

2022-02-02 Thread Marios Trivyzas (Jira)
Marios Trivyzas created FLINK-25928: --- Summary: Refactor timestamp<->number validation messages Key: FLINK-25928 URL: https://issues.apache.org/jira/browse/FLINK-25928 Project: Flink Issue T

[jira] [Created] (FLINK-25927) Make flink-connector-base dependency usage consistent across all connectors.

2022-02-02 Thread Alexander Fedulov (Jira)
Alexander Fedulov created FLINK-25927: - Summary: Make flink-connector-base dependency usage consistent across all connectors. Key: FLINK-25927 URL: https://issues.apache.org/jira/browse/FLINK-25927

[jira] [Created] (FLINK-25926) Update org.postgresql:postgresql to 42.3.2

2022-02-02 Thread Martijn Visser (Jira)
Martijn Visser created FLINK-25926: -- Summary: Update org.postgresql:postgresql to 42.3.2 Key: FLINK-25926 URL: https://issues.apache.org/jira/browse/FLINK-25926 Project: Flink Issue Type: Te

[jira] [Created] (FLINK-25925) JobMasterTest.testJobMasterAcceptsExcessSlotsWhenJobIsRestarting

2022-02-02 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-25925: - Summary: JobMasterTest.testJobMasterAcceptsExcessSlotsWhenJobIsRestarting Key: FLINK-25925 URL: https://issues.apache.org/jira/browse/FLINK-25925 Project: Flink

[jira] [Created] (FLINK-25924) KDF Integration tests intermittently fails

2022-02-02 Thread Zichen Liu (Jira)
Zichen Liu created FLINK-25924: -- Summary: KDF Integration tests intermittently fails Key: FLINK-25924 URL: https://issues.apache.org/jira/browse/FLINK-25924 Project: Flink Issue Type: New Featur

[jira] [Created] (FLINK-25923) Add tests for native savepoint format schema evolution

2022-02-02 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-25923: Summary: Add tests for native savepoint format schema evolution Key: FLINK-25923 URL: https://issues.apache.org/jira/browse/FLINK-25923 Project: Flink

[jira] [Created] (FLINK-25922) KinesisFirehoseSinkITCase hangs on AZP

2022-02-02 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-25922: - Summary: KinesisFirehoseSinkITCase hangs on AZP Key: FLINK-25922 URL: https://issues.apache.org/jira/browse/FLINK-25922 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-25921) Support different input parallelism for preCommit topology

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25921: --- Summary: Support different input parallelism for preCommit topology Key: FLINK-25921 URL: https://issues.apache.org/jira/browse/FLINK-25921 Project: Flink Issu

[jira] [Created] (FLINK-25920) Allow receiving updates of CommittableSummary

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25920: --- Summary: Allow receiving updates of CommittableSummary Key: FLINK-25920 URL: https://issues.apache.org/jira/browse/FLINK-25920 Project: Flink Issue Type: Sub-t

[jira] [Created] (FLINK-25919) Sink V2 improvements and followups

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25919: --- Summary: Sink V2 improvements and followups Key: FLINK-25919 URL: https://issues.apache.org/jira/browse/FLINK-25919 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-25918) Use FileEnumerator to implement filter pushdown of filepath metadata

2022-02-02 Thread Francesco Guardiani (Jira)
Francesco Guardiani created FLINK-25918: --- Summary: Use FileEnumerator to implement filter pushdown of filepath metadata Key: FLINK-25918 URL: https://issues.apache.org/jira/browse/FLINK-25918 P

[jira] [Created] (FLINK-25917) Share RpcSystem aross tests

2022-02-02 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-25917: Summary: Share RpcSystem aross tests Key: FLINK-25917 URL: https://issues.apache.org/jira/browse/FLINK-25917 Project: Flink Issue Type: Technical Deb

Re: [ANNOUNCE] Apache Flink Stateful Functions 3.2.0 released

2022-02-02 Thread Etienne Chauchot
Congrats to everyone involved ! Best Etienne Le 01/02/2022 à 12:23, Till Rohrmann a écrit : The Apache Flink community is very happy to announce the release of Apache Flink Stateful Functions 3.2.0. Stateful Functions is an API that simplifies building distributed stateful applications. It's