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

2023-04-12 Thread Lincoln Lee
Hi, Mang +1 for completing the support for atomicity of CTAS, this is very useful in batch scenarios. I have two questions: 1. naming wise: a) can we rename the `Catalog#getTwoPhaseCommitCreateTable` to `Catalog#twoPhaseCreateTable` (and we may add

Re: [DISCUSS] FLIP-304: Pluggable failure handling for Apache Flink

2023-04-12 Thread Leonard Xu
Thanks Panagiotis for the update, the updated FLIP looks good to me. Best, Leonard > On Apr 13, 2023, at 7:42 AM, Panagiotis Garefalakis wrote: > > Hello there, > > Zhu: agree with the config option, great suggestion > Hong: global timeout is also interesting and a good addition -- only >

Re: [DISCUSS] FLIP-302: Support TRUNCATE TABLE statement

2023-04-12 Thread liu ron
Hi, xia Thanks for your explanation, for the first question, given the current status, I think we can provide the generic interface in the future if we need it. For the second question, it makes sense to me if we can support the table cache at the framework level. Best, Ron yuxia

[jira] [Created] (FLINK-31791) FLIP-292: Enhance COMPILED PLAN to support operator-level state TTL configuration

2023-04-12 Thread Jane Chan (Jira)
Jane Chan created FLINK-31791: - Summary: FLIP-292: Enhance COMPILED PLAN to support operator-level state TTL configuration Key: FLINK-31791 URL: https://issues.apache.org/jira/browse/FLINK-31791 Project:

Re: [DISCUSS] Release Paimon 0.4

2023-04-12 Thread Jingsong Li
Oh sorry to Flink devs, This email should be sent to paimon dev, please ignore it. Best, Jingsong On Thu, Apr 13, 2023 at 10:56 AM Jingsong Li wrote: > > Hi everyone, > > I'm going to check out the 0.4 branch out next Monday, and won't merge > major refactoring-related PRs into master branch

[DISCUSS] Release Paimon 0.4

2023-04-12 Thread Jingsong Li
Hi everyone, I'm going to check out the 0.4 branch out next Monday, and won't merge major refactoring-related PRs into master branch until next Monday. Blockers: - Entire Database Sync CC @Caizhi Weng - CDC Ingestion mysql DATETIME(6) cast error [1] - MySqlSyncTableAction should support case

[RESULT][VOTE] FLIP-292: Enhance COMPILED PLAN to support operator-level state TTL configuration

2023-04-12 Thread Jane Chan
Hi everyone, FLIP-292 [1] has been accepted and voted through this thread [2]. There are seven approving votes, six of which are binding: - Lincoln Lee (binding) - Jing Ge (binding) - godfrey he (binding) - Jark Wu (binding) - Shuo Cheng - Benchao Li (binding) - Leonard Xu

Re: [mongodb connector] which flink version(s) is the mongodb connector compatible with?

2023-04-12 Thread Jiabao Sun
Hi Saketh Hi Saketh, As yuxia said, currently the mongodb connector 1.0.0 is compatible with flink 1.16. Some compatibility fixes have been made recently to make it compatible with flink 1.17, but it has not been released yet. Best, Jiabao > 2023年4月13日 上午9:36,yuxia 写道: > > Oh, correct

[jira] [Created] (FLINK-31790) Filesystem batch sink should also respect to the PartitionCommitPolicy

2023-04-12 Thread Aitozi (Jira)
Aitozi created FLINK-31790: -- Summary: Filesystem batch sink should also respect to the PartitionCommitPolicy Key: FLINK-31790 URL: https://issues.apache.org/jira/browse/FLINK-31790 Project: Flink

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

2023-04-12 Thread liu ron
Hi, Mang Atomicity is very important for CTAS, especially for batch jobs. This FLIP is a continuation of FLIP-218, which is valuable for CTAS. I just have one question, in the Motivation part of FLIP-218, we mentioned three levels of atomicity semantics, can this current design do the same as

[DISCUSS] EncodingFormat and DecondingFormat provide copy API

2023-04-12 Thread tanjialiang
Hi, devs. I'd like to start a discussion about to EncodingFormat and DecondingFormat provide copy API, which relate to FLINK-31686 [1]. Current, DecodingFormat doesn't support copy(), which makes the DecodingFormat resued after filter/projection is pushed down. The EncodingFormat has the

Re: [mongodb connector] which flink version(s) is the mongodb connector compatible with?

2023-04-12 Thread yuxia
Oh, correct what I said. The version of released connector jars on Maven central has the supported Flink version as suffix. For the case of the mongodb connector with version 1.0.0-1.16[1], it should be compatible with Flink 1.16. [1]

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-12 Thread Alexander Sorokoumov
Hi Jark, John, Thank you for the discussion! I will proceed with completing the patch that adds exactly-once to upsert-kafka connector. Best, Alexander On Wed, Apr 12, 2023 at 12:21 AM Jark Wu wrote: > Hi John, > > Thank you for your valuable input. It sounds reasonable to me. > > From this

Re: [mongodb connector] which flink version(s) is the mongodb connector compatible with?

2023-04-12 Thread yuxia
>From the pom[1] of the connector, it seems it's compatible with Flink 1.17. [1] https://github.com/apache/flink-connector-mongodb/blob/main/pom.xml#L56 Best regards, Yuxia - 原始邮件 - 发件人: "Saketh Kurnool" 收件人: "dev" 发送时间: 星期四, 2023年 4 月 13日 上午 5:27:06 主题: [mongodb connector] which

Re: [DISCUSS] FLIP 295: Support persistence of Catalog configuration and asynchronous registration

2023-04-12 Thread Shammon FY
Hi Feng Thanks for your update. I found there are two options in `Proposed Changes`, can you put the unselected option in `Rejected Alternatives`? I think this may help us better understand your proposal Best, Shammon FY On Thu, Apr 13, 2023 at 4:49 AM Jing Ge wrote: > Hi Feng, > > Thanks

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

2023-04-12 Thread Tzu-Li (Gordon) Tai
RC2 (for Flink 1.17.0) vote has started in a separate thread: https://lists.apache.org/thread/mff76c2hzcb1mk8fm5m2h4z0j73qz2vk Please test and cast your votes! On Tue, Apr 11, 2023 at 11:45 AM Martijn Visser wrote: > +1, thanks for driving this Gordon. > > On Tue, Apr 11, 2023 at 8:15 PM

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

2023-04-12 Thread Tzu-Li (Gordon) Tai
A few important remarks about this release candidate: - As mentioned in the previous voting thread of RC1 [1], we've decided to skip releasing a version of the externalized Flink Kafka Connector matching with Flink 1.16.x since the original vote thread stalled, and meanwhile we've already

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

2023-04-12 Thread Tzu-Li (Gordon) Tai
Hi everyone, Please review and vote on release candidate #2 for version 3.0.0 of the Apache Flink Kafka Connector, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which

Re: [DISCUSS] FLIP-304: Pluggable failure handling for Apache Flink

2023-04-12 Thread Panagiotis Garefalakis
Hello there, Zhu: agree with the config option, great suggestion Hong: global timeout is also interesting and a good addition -- only downside I see is just another config option If everyone is happy, I suggest we keep the discussion open until Friday and start a Vote shortly after. Cheers,

[mongodb connector] which flink version(s) is the mongodb connector compatible with?

2023-04-12 Thread Saketh Kurnool
Hello! I have a use case for the mongodb connector (see: https://github.com/apache/flink-connector-mongodb), but I am currently running flink 1.15. As far as I can tell, the connector relies on dependencies that don't exist in 1.15 (one such example is

Re: [DISCUSS] FLIP 295: Support persistence of Catalog configuration and asynchronous registration

2023-04-12 Thread Jing Ge
Hi Feng, Thanks for raising this FLIP. I am still confused after completely reading the thread with following questions: 1. Naming confusion - registerCatalog() and addCatalog() have no big difference based on their names. One of them is responsible for data persistence. How about

[jira] [Created] (FLINK-31789) Update axel to current

2023-04-12 Thread Morey Straus (Jira)
Morey Straus created FLINK-31789: Summary: Update axel to current Key: FLINK-31789 URL: https://issues.apache.org/jira/browse/FLINK-31789 Project: Flink Issue Type: Bug Affects Versions:

[jira] [Created] (FLINK-31788) Add back Support emitValueWithRetract for TableAggregateFunction

2023-04-12 Thread Feng Jin (Jira)
Feng Jin created FLINK-31788: Summary: Add back Support emitValueWithRetract for TableAggregateFunction Key: FLINK-31788 URL: https://issues.apache.org/jira/browse/FLINK-31788 Project: Flink

Re: [DISCUSS] Add support for Apache Arrow format

2023-04-12 Thread Martijn Visser
Which connectors would be commonly used when reading in Arrow format? Filesystem? On Wed, Apr 12, 2023 at 4:27 AM Jacky Lau wrote: > Hi >I also think arrow format will be useful when reading/writing with > message queue. >Arrow defines a language-independent columnar memory format for

[jira] [Created] (FLINK-31787) Add the explicit ROW constructor to the system function doc

2023-04-12 Thread Aitozi (Jira)
Aitozi created FLINK-31787: -- Summary: Add the explicit ROW constructor to the system function doc Key: FLINK-31787 URL: https://issues.apache.org/jira/browse/FLINK-31787 Project: Flink Issue Type:

[jira] [Created] (FLINK-31786) Removing unused HighAvailabilityServices implementations

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31786: - Summary: Removing unused HighAvailabilityServices implementations Key: FLINK-31786 URL: https://issues.apache.org/jira/browse/FLINK-31786 Project: Flink

[jira] [Created] (FLINK-31785) Move LeaderElectionService out of LeaderContender

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31785: - Summary: Move LeaderElectionService out of LeaderContender Key: FLINK-31785 URL: https://issues.apache.org/jira/browse/FLINK-31785 Project: Flink Issue

[jira] [Created] (FLINK-31784) Add multiple-component support to DefaultLeaderElectionService

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31784: - Summary: Add multiple-component support to DefaultLeaderElectionService Key: FLINK-31784 URL: https://issues.apache.org/jira/browse/FLINK-31784 Project: Flink

[jira] [Created] (FLINK-31783) Replace LeaderElectionDriver in DefaultLeaderElectionService with MultipleComponentLeaderElectionService

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31783: - Summary: Replace LeaderElectionDriver in DefaultLeaderElectionService with MultipleComponentLeaderElectionService Key: FLINK-31783 URL:

[jira] [Created] (FLINK-31781) Introduce contender ID into LeaderElectionService interface

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31781: - Summary: Introduce contender ID into LeaderElectionService interface Key: FLINK-31781 URL: https://issues.apache.org/jira/browse/FLINK-31781 Project: Flink

[jira] [Created] (FLINK-31782) Make DefaultLeaderElectionService implement MultipleComponentLeaderElectionService.Listener

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31782: - Summary: Make DefaultLeaderElectionService implement MultipleComponentLeaderElectionService.Listener Key: FLINK-31782 URL: https://issues.apache.org/jira/browse/FLINK-31782

[jira] [Created] (FLINK-31780) Allow users to enable Ensemble tracking for ZooKeeper

2023-04-12 Thread Oleksandr Nitavskyi (Jira)
Oleksandr Nitavskyi created FLINK-31780: --- Summary: Allow users to enable Ensemble tracking for ZooKeeper Key: FLINK-31780 URL: https://issues.apache.org/jira/browse/FLINK-31780 Project: Flink

[jira] [Created] (FLINK-31779) Track stable branch of externalized connector instead of specific release tag

2023-04-12 Thread Weijie Guo (Jira)
Weijie Guo created FLINK-31779: -- Summary: Track stable branch of externalized connector instead of specific release tag Key: FLINK-31779 URL: https://issues.apache.org/jira/browse/FLINK-31779 Project:

[jira] [Created] (FLINK-31778) Casting array of rows produces incorrect result

2023-04-12 Thread Ilya Soin (Jira)
Ilya Soin created FLINK-31778: - Summary: Casting array of rows produces incorrect result Key: FLINK-31778 URL: https://issues.apache.org/jira/browse/FLINK-31778 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-31777) Upsert Kafka use Avro Confluent, key is ok, but all values are null.

2023-04-12 Thread Alvin Ge (Jira)
Alvin Ge created FLINK-31777: Summary: Upsert Kafka use Avro Confluent, key is ok, but all values are null. Key: FLINK-31777 URL: https://issues.apache.org/jira/browse/FLINK-31777 Project: Flink

[jira] [Created] (FLINK-31776) Introducing sub-interface LeaderElectionService.LeaderElection

2023-04-12 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-31776: - Summary: Introducing sub-interface LeaderElectionService.LeaderElection Key: FLINK-31776 URL: https://issues.apache.org/jira/browse/FLINK-31776 Project: Flink

Re: [VOTE] Release flink-connector-aws, 4.1.0 for Flink 1.17

2023-04-12 Thread Samrat Deb
+1 (non binding) On Tue, 4 Apr 2023 at 1:42 AM, Elphas Tori wrote: > +1 (non-binding) > > + verified hashes and signatures > + checked local build of website pull request and approved > > On 2023/04/03 16:19:00 Danny Cranmer wrote: > > Hi everyone, > > Please review and vote on the binaries

Re: [DISCUSS] FLIP-306: Unified File Merging Mechanism for Checkpoints

2023-04-12 Thread Zakelly Lan
Hi Yun, I reorganized our discussion and added a comparison table of state ownership with some previous designs. Please take a look at section "4.9. State ownership comparison with other designs". But I don't see them as alternatives since the design of state ownership is integrated with this

Re: [VOTE] Release flink-connector-aws, 4.1.0 for Flink 1.17

2023-04-12 Thread Robert Metzger
+1 (binding) - tried out the kinesis sql binary with sql client - staging binaries look fine On Mon, Apr 3, 2023 at 10:12 PM Elphas Tori wrote: > +1 (non-binding) > > + verified hashes and signatures > + checked local build of website pull request and approved > > On 2023/04/03 16:19:00 Danny

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-12 Thread Jark Wu
Hi John, Thank you for your valuable input. It sounds reasonable to me. >From this point of view, the exactly-once is used to guarantee transaction semantics other than avoid duplication/upserts. This is similar to the JDBC connectors that already support eventual consistency with idempotent

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

2023-04-12 Thread Hongshun Wang
Hi everyone, I have already modified FLIP-288 to provide a newDiscoveryOffsetsInitializer in the KafkaSourceBuilder and KafkaSourceEnumerator. Users can use KafkaSourceBuilder#setNewDiscoveryOffsets to change the strategy for new partitions. Surely, enabling the partition discovery strategy by