Re: [DISCUSS] FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source

2023-04-18 Thread Leonard Xu
Thanks Hongshun for deeper analysis of the existing KafkaSource implementation details, Cool! There’s no specific use case to use a future TIMESTAMP and SPECIFIC-OFFSET for new discovered partitions The existing SpecifiedOffsetsInitializer will use the EARLIEST offset for unspecified

Re:Re: [DISCUSS] FLIP-305: Support atomic for CREATE TABLE AS SELECT(CTAS) statement

2023-04-18 Thread Mang Zhang
Hi, Yuxia Thank you for your reply. We can identify whether a CatalogTable supports atomic Ctas by determining its type in DynamicTableFactory/DynamicTableSink, like the following: boolean isAtomicCtas = context.getCatalogTable().getOrigin() instanceof TwoPhaseCatalogTable; And I've updated the

[RESULT] [VOTE] Apache Flink ML Release 2.2.0, release candidate #2

2023-04-18 Thread Dong Lin
I'm happy to announce that we have unanimously approved this release [1]. There are 4 approving votes, 3 of which are binding: - Guowei Ma (binding) - Xingbo Huang (binding) - Dong Lin (binding) - Zhipeng Zhang (non-binding) There are no disapproving votes. Thank you for verifying the release

[discuss] FLINK-25132: KafkaSource cannot work with object-reusing DeserializationSchema result in a performance regression

2023-04-18 Thread 何军
Hi everyone, I would like to start a discussion on FLINK-25132: KafkaSource cannot work with object-reusing DeserializationSchema. The implementation of FLINK-25132 moves the deserialization of kafka records from SplitFetcher thread to thread that runs KafkaRecordEmitter, namely flink's main

Re: [DISCUSS] FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source

2023-04-18 Thread Hongshun Wang
Hi Shammon, I agree with you. Since only EARLIEST is used, it's better not to mislead users through the interface. Yours Hongshun On Tue, Apr 18, 2023 at 7:12 PM Shammon FY wrote: > Hi Hongshun > > Thanks for your explanation, I have got your point. I review the FLIP again > and only have

[jira] [Created] (FLINK-31846) Support cancel final checkpoint when all tasks are finished

2023-04-18 Thread Fan Hong (Jira)
Fan Hong created FLINK-31846: Summary: Support cancel final checkpoint when all tasks are finished Key: FLINK-31846 URL: https://issues.apache.org/jira/browse/FLINK-31846 Project: Flink Issue

Re: [DISCUSS] Status of Statefun Project

2023-04-18 Thread Marco Villalobos
I am currently using Stateful Functions in my application. I use Apache Flink for stream processing, and StateFun as a hand-off point for the rest of the application. It serves well as a bridge between a Flink Streaming job and micro-services. I would be disappointed if StateFun was sunsetted.

Re: [Discussion] - Take findify/flink-scala-api under Flink umbrella

2023-04-18 Thread Alexey Novakov
Hi Martijn, Thanks for your reply and attention. 1. As I read Nick's report here https://issues.apache.org/jira/browse/FLINK-13414?focusedCommentId=17257763=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17257763 Scala maintainers were blocked by Flink's source code

Re: [Discussion] - Take findify/flink-scala-api under Flink umbrella

2023-04-18 Thread Alexey Novakov
Hi Günter, David, Let me reply to you both in one email. First of all, thank you for engaging. Günter: - I fully agree that losing Scala API as officially supported in Flink would be very unfortunate. Future of Scala is interesting and will bring more benefits to Flink users. Just to remind

Re: [VOTE] Release flink-connector-kafka 3.0.0 for Flink 1.17, release candidate #2

2023-04-18 Thread Tzu-Li (Gordon) Tai
+1 (binding) - Checked hashes and signatures - Built from source mvn clean install -Pcheck-convergence -Dflink.version=1.17.0 - Eyeballed NOTICE license files - Started a Flink 1.17.0 cluster + Kafka 3.2.3 cluster, submitted a SQL statement using the Kafka connector under exactly-once mode.

Re: [VOTE] Release flink-connector-opensearch, release candidate #1

2023-04-18 Thread Ryan Skraba
Hello! +1 (non-binding) I've validated the source for the RC1: flink-connector-opensearch-1.0.1-src.tgz * The sha512 checksum is OK. * The source file is signed correctly. * The signature A5F3BCE4CBE993573EC5966A65321B8382B219AF is found in the KEYS file, and on https://keys.openpgp.org * The

[jira] [Created] (FLINK-31845) Make KubernetesStepDecorator Pluggable

2023-04-18 Thread Daren Wong (Jira)
Daren Wong created FLINK-31845: -- Summary: Make KubernetesStepDecorator Pluggable Key: FLINK-31845 URL: https://issues.apache.org/jira/browse/FLINK-31845 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-31844) DefaultPackagedProgramRetriever#getClasspathsFromUserLibDir supports loading profile

2023-04-18 Thread Bo Cui (Jira)
Bo Cui created FLINK-31844: -- Summary: DefaultPackagedProgramRetriever#getClasspathsFromUserLibDir supports loading profile Key: FLINK-31844 URL: https://issues.apache.org/jira/browse/FLINK-31844 Project:

[jira] [Created] (FLINK-31843) Select slots from SlotPool#freeSlots in bulk

2023-04-18 Thread Weihua Hu (Jira)
Weihua Hu created FLINK-31843: - Summary: Select slots from SlotPool#freeSlots in bulk Key: FLINK-31843 URL: https://issues.apache.org/jira/browse/FLINK-31843 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-31842) calculate task executor's utilization only when used

2023-04-18 Thread Weihua Hu (Jira)
Weihua Hu created FLINK-31842: - Summary: calculate task executor's utilization only when used Key: FLINK-31842 URL: https://issues.apache.org/jira/browse/FLINK-31842 Project: Flink Issue Type:

[jira] [Created] (FLINK-31841) Redundant local variables in AllWindowedStream#reduce

2023-04-18 Thread Wencong Liu (Jira)
Wencong Liu created FLINK-31841: --- Summary: Redundant local variables in AllWindowedStream#reduce Key: FLINK-31841 URL: https://issues.apache.org/jira/browse/FLINK-31841 Project: Flink Issue

[jira] [Created] (FLINK-31840) NullPointerException in operators.window.slicing.SliceAssigners$AbstractSliceAssigner.assignSliceEnd

2023-04-18 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-31840: -- Summary: NullPointerException in operators.window.slicing.SliceAssigners$AbstractSliceAssigner.assignSliceEnd Key: FLINK-31840 URL:

[jira] [Created] (FLINK-31839) It's not possible to add flink-s3-fs-hadoop and flink-s3-fs-presto plugins at the same time

2023-04-18 Thread Gabor Somogyi (Jira)
Gabor Somogyi created FLINK-31839: - Summary: It's not possible to add flink-s3-fs-hadoop and flink-s3-fs-presto plugins at the same time Key: FLINK-31839 URL: https://issues.apache.org/jira/browse/FLINK-31839

[jira] [Created] (FLINK-31838) Move thread handling from DefaultMultipleComponentLeaderElectionService into DefaultLeaderElectionService

2023-04-18 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31838: - Summary: Move thread handling from DefaultMultipleComponentLeaderElectionService into DefaultLeaderElectionService Key: FLINK-31838 URL:

Re: [DISCUSS] FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source

2023-04-18 Thread Shammon FY
Hi Hongshun Thanks for your explanation, I have got your point. I review the FLIP again and only have one minor comment which won't block this FLIP: should we need in `OffsetsInitializer newDiscoveryOffsetsInitializer` in the constructor of `KafkaSourceEnumerator`? I think we can remove it if we

[jira] [Created] (FLINK-31837) Move LeaderElectionDriver instantiated into DefaultLeaderElectionService's constructor

2023-04-18 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31837: - Summary: Move LeaderElectionDriver instantiated into DefaultLeaderElectionService's constructor Key: FLINK-31837 URL: https://issues.apache.org/jira/browse/FLINK-31837

[jira] [Created] (FLINK-31836) Upgrade to Calcite version to 1.34.0

2023-04-18 Thread Sergey Nuyanzin (Jira)
Sergey Nuyanzin created FLINK-31836: --- Summary: Upgrade to Calcite version to 1.34.0 Key: FLINK-31836 URL: https://issues.apache.org/jira/browse/FLINK-31836 Project: Flink Issue Type:

Re: Re: [VOTE] FLIP-302: Support TRUNCATE TABLE statement in batch mode

2023-04-18 Thread Benchao Li
+1 (binding) Hang Ruan 于2023年4月18日周二 14:03写道: > +1 (no-binding) > > Best, > Hang > > Shammon FY 于2023年4月18日周二 13:33写道: > > > +1 (no-binding) > > > > Best, > > Shammon FY > > > > On Tue, Apr 18, 2023 at 12:56 PM Jacky Lau wrote: > > > > > +1 (no-binding) > > > > > > Best, > > > Jacky Lau > > >

[jira] [Created] (FLINK-31835) DataTypeHint don't support Row>

2023-04-18 Thread jeff-zou (Jira)
jeff-zou created FLINK-31835: Summary: DataTypeHint don't support Row> Key: FLINK-31835 URL: https://issues.apache.org/jira/browse/FLINK-31835 Project: Flink Issue Type: Bug

Re: [DISCUSS] FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source

2023-04-18 Thread Hongshun Wang
Hi Shammon, Thank you for your advice.I have carefully considered whether to show this in SQL DDL. Therefore, I carefully studied whether it is feasible Recently However, after reading the corresponding code more thoroughly, it appears that SpecifiedOffsetsInitializer and

Re: Need Help with Slack Invite Link

2023-04-18 Thread Martijn Visser
Hi Yun, You can also set the link to not expire, that's what I've always done. By the way, the docs already have an updated Slack invite link. Best regards, Martijn On Mon, Apr 17, 2023 at 6:04 PM Yun Tang wrote: > Hi Madhur, > > Could you try this invitation link: >

[jira] [Created] (FLINK-31834) Azure Warning: no space left on device

2023-04-18 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-31834: -- Summary: Azure Warning: no space left on device Key: FLINK-31834 URL: https://issues.apache.org/jira/browse/FLINK-31834 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-31833) Support code-gen fusion for multiple operators

2023-04-18 Thread Godfrey He (Jira)
Godfrey He created FLINK-31833: -- Summary: Support code-gen fusion for multiple operators Key: FLINK-31833 URL: https://issues.apache.org/jira/browse/FLINK-31833 Project: Flink Issue Type:

[jira] [Created] (FLINK-31832) Add benchmarks for end to end  restarting tasks

2023-04-18 Thread Weihua Hu (Jira)
Weihua Hu created FLINK-31832: - Summary: Add benchmarks for end to end  restarting tasks Key: FLINK-31832 URL: https://issues.apache.org/jira/browse/FLINK-31832 Project: Flink Issue Type:

Re: Re: [VOTE] FLIP-302: Support TRUNCATE TABLE statement in batch mode

2023-04-18 Thread Hang Ruan
+1 (no-binding) Best, Hang Shammon FY 于2023年4月18日周二 13:33写道: > +1 (no-binding) > > Best, > Shammon FY > > On Tue, Apr 18, 2023 at 12:56 PM Jacky Lau wrote: > > > +1 (no-binding) > > > > Best, > > Jacky Lau > > > > Jingsong Li 于2023年4月18日周二 11:57写道: > > > > > +1 > > > > > > On Tue, Apr 18,