Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-10-10 Thread Hongshun Wang
ption. It is OK to add an info level logging if we want to. It is a > clear intention if the user has explicitly changed subscription and > restarted the job. There is no need to add a config to double confirm. > > Regards, > > Jiangjie (Becket) Qin > > On Thu, Oct 9, 2

Re: Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-26 Thread Hongshun Wang
der.addReader() is called for the recovered reader. When the > SplitEnumerator.addSplitsBack() is invoked, it is for sure a partial > recovery. And the enumerator should remove these splits from the split > assignment map as if they were never assigned. > > I think this should work,

Re: Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-25 Thread Hongshun Wang
bove pattern. Do we really need it? > > Thanks, > > Jiangjie (Becket) Qin > > On Mon, Sep 8, 2025 at 8:18 PM Hongshun Wang > wrote: > > > Hi Becket, > > > > > I am curious what would the enumerator do differently for the splits > > added via addSp

Re: [VOTE] Apache Flink CDC Release 3.5.0, release candidate #3

2025-09-23 Thread Hongshun Wang
Thanks Yanquan for driving this release! +1 (non-binding) I have test Yaml job which read from Postgres to Fluss. It works well but some documents error: 1. conf/flink-conf.yaml. -> conf/conf.yaml 2. type: posgtres -> postgres 3. The value of option `tables` is `adb.\.*.\.*, bdb.user_schema_[0

[jira] [Created] (FLINK-38335) SplitEnumeratorContext supports fixed delay async call.

2025-09-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38335: - Summary: SplitEnumeratorContext supports fixed delay async call. Key: FLINK-38335 URL: https://issues.apache.org/jira/browse/FLINK-38335 Project: Flink

SplitEnumeratorContext supports fixed delay async call.

2025-09-10 Thread Hongshun Wang
Hi devs, I'd like to start a discussion on adding support for fixed-delay asynchronous calls in SplitEnumeratorContext [1]. Because it's a public API, I'm interested in your thoughts: 1. Is this functionality needed in the current design? 2. Would this require a small FLIP or a jira is enough. C

RE: Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-08 Thread Hongshun Wang
itReassignmentOnRecovery", > right? > > Thanks, > > Jiangjie (Becket) Qin > > On Mon, Sep 8, 2025 at 1:48 AM Hongshun Wang > wrote: > > > Hi devs, > > > > It has been quite some time since this FLIP[1] was first proposed. Thank > > you for

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-08 Thread Hongshun Wang
tomorrow. Best Hongshun [1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=373886480 On Mon, Sep 8, 2025 at 4:42 PM Hongshun Wang wrote: > Hi Leonard, > Thanks for your advice. It makes sense and I have modified it. > > Best, > Hongshun > > On Mon, Se

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-08 Thread Hongshun Wang
y, we can provide better backward compatibility > and also makes it easier for developers to understand. > > Best, > Leonard > > > > 2025 9月 3 20:26,Hongshun Wang 写道: > > Hi Becket, > > I think that's a great idea! I have added the > SupportSplitReassignmentOnRecov

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-09-03 Thread Hongshun Wang
. > > Thanks, > > Jiangjie (Becket) Qin > > On Wed, Aug 20, 2025 at 8:52 PM Hongshun Wang > wrote: > >> Hi de vs, >> >> Would anyone like to discuss this FLIP? I'd appreciate your feedback and >> suggestions. >> >> Best, >> Hongshu

Re: [VOTE] FLIP-535: Introduce RateLimiter to Source

2025-09-03 Thread Hongshun Wang
+1 (no binding) Best, Hongshun > 2025年9月3日 17:06,Yanquan Lv 写道: > > +1(binding), and I am committed to driving forward the subsequent work > items. > > Best, > Yanquan > > Jiangang Liu 于2025年9月3日周三 15:59写道: > >> +1 (non-binding). >> >> Best Regards, >> Jiangang Liu >> >> Xiqian YU 于2025年

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-08-20 Thread Hongshun Wang
Hi de vs, Would anyone like to discuss this FLIP? I'd appreciate your feedback and suggestions. Best, Hongshun > 2025年8月13日 14:23,Hongshun Wang 写道: > > Hi Becket, > Thank you for your detailed feedback. The new contract makes good sense to me > and effectively ad

[jira] [Created] (FLINK-38265) Stream Split shouldn't finish when exception occors

2025-08-18 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38265: - Summary: Stream Split shouldn't finish when exception occors Key: FLINK-38265 URL: https://issues.apache.org/jira/browse/FLINK-38265 Project: Flink

Re: [DISCUSS] Planning Flink 2.2

2025-08-13 Thread Hongshun Wang
+1 from my side. Thanks Hang for driving this. Best, Hongshun On Wed, Aug 13, 2025 at 11:45 PM Weiqing Yang wrote: > Thanks Hang for kicking off the discussion and volunteering as release > manager. +1 for the proposed feature freeze date and release timeline. > > Cheers, > Weiqing > > On Tue

Re: [ANNOUNCE] New Apache Flink PMC Member - Ron Liu

2025-08-13 Thread Hongshun Wang
Congrats, Ron! You deserve it! Best, Hongshun On Thu, Aug 14, 2025 at 2:03 PM Geng Biao wrote: > Congrats, Ron! > > 发送自 Outlook for iOS > > 发件人: Zakelly Lan > 发送时间: Thursday, August 14, 2025 11:26:00 AM > 收件人: dev@flink.apache.org > 主题:

[jira] [Created] (FLINK-38244) When the table name of mysql supports size sensitive, fix the error that column name not being found

2025-08-13 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38244: - Summary: When the table name of mysql supports size sensitive, fix the error that column name not being found Key: FLINK-38244 URL: https://issues.apache.org/jira/browse/FLINK

[jira] [Created] (FLINK-38243) When the table name of mysql supports size sensitive, fix the error that column name not being found

2025-08-13 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38243: - Summary: When the table name of mysql supports size sensitive, fix the error that column name not being found Key: FLINK-38243 URL: https://issues.apache.org/jira/browse/FLINK

Re: [VOTE] Release flink-connector-kafka v4.0.1, release candidate #2

2025-08-13 Thread Hongshun Wang
Hi Faibian, I have a question unrelated to this release: why isn't flink-streaming-kafka-test-base available in the official Maven repository? I've come across several unofficial versions published by third parties[1], but there doesn't seem to be a centralized, official artifact. This makes it d

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-08-12 Thread Hongshun Wang
restarts. > 4. When reader A recovers, it will again report splits (1 and 2) to the > enumerator. > 5. The enumerator should ignore this report because it has assigned splits > (1 and 2) to reader B. > > So with the new contract, the enumerator should be the source of truth for >

Re: [DISCUSS] FLIP-529 Connections in Flink SQL and Table API

2025-08-06 Thread Hongshun Wang
Hi Hao, > System connection is not bound to catalog and can be used to create catalog. It looks good to me. Best, Hongshun > 2025年8月7日 00:05,Hao Li 写道: > > System connection is > not bound to catalog and can be used to create catalog.

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-08-06 Thread Hongshun Wang
ion to remember the initially > assigned splits, if it wants to wait until all the readers are registered. > This also allow future addition of reader information. > > Thanks, > > Jiangjie (Becket) Qin > > > > On Mon, Aug 4, 2025 at 8:39 PM Hongshun Wang > wrote:

[DISCUSS] FLIP-529 Connections in Flink SQL and Table API

2025-08-05 Thread Hongshun Wang
Hi Mayank, Sorry for joining this FLIP discussion late. I can see that it significantly reduces complexity and improves security when creating tables. However, I have a question regarding its usage in the context of catalogs: In the current FLIP, when creating a connection within a catalog, we u

Re: [DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-08-04 Thread Hongshun Wang
age, current > design is a nice tradeoff and has considered the new Source implementation > details, +1 from my side. > > > Best, > Leonard > > > > > 2025 7月 19 18:59,Hongshun Wang 写道: > > > > Hi devs, > > > > I'd like to initiate a discussion

[jira] [Created] (FLINK-38184) CDC no need to getCopyOfBuffer for each split info.

2025-08-03 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38184: - Summary: CDC no need to getCopyOfBuffer for each split info. Key: FLINK-38184 URL: https://issues.apache.org/jira/browse/FLINK-38184 Project: Flink Issue

Re: [VOTE] Release 2.1.0, release candidate #1

2025-07-27 Thread Hongshun Wang
+1 (no binding) - verified & checksums - use release-2.1.0-rc1.tar.gz to run sql jobs. Best, Hongshun On Mon, Jul 28, 2025 at 11:27 AM Leonard Xu wrote: > +1 (binding) > > - verified signatures > - verified hashsums > - checked Github release tag > - checked release notes > - ran SQL values te

Re: [DISCUSS] State Schema Evolution for RowData

2025-07-23 Thread Hongshun Wang
Hi Weiqing, I like the idea. Would you give an example how a kafka or mysql connector uses it to read data with different schemas for better understanding? Best, Hongshun On Thu, Jul 24, 2025 at 10:34 AM Shengkai Fang wrote: > Hi, Weiqing. > > Thanks for the update. It's better you can update t

Re: [DISCUSS] Projection pushdown support in Flink Kafka Table connector

2025-07-21 Thread Hongshun Wang
Hi Farooq, Projection pushdown is often used to pushdown projection from flink to storage server. However, Kafka is a row format, and kafka broker not support it. Thus, the pushdown only be applied in kafka source. It will make the connector more complex, will it improve a lot? Best Hongshun On T

[DISCUSS] FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split assignment

2025-07-19 Thread Hongshun Wang
Hi devs, I'd like to initiate a discussion about [FLIP-537: Enumerator with Global Split Assignment Distribution for Balanced Split Assignment] [1], which addresses critical limitations in our current Kafka connector split distribution mechanism. As documented in [FLINK-31762] [2], several scenar

[jira] [Created] (FLINK-38030) Table.explainSql throw exception: xxx cannot be cast to class org.apache.flink.table.catalog.AbstractCatalog

2025-06-29 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-38030: - Summary: Table.explainSql throw exception: xxx cannot be cast to class org.apache.flink.table.catalog.AbstractCatalog Key: FLINK-38030 URL: https://issues.apache.org/jira

Re: [VOTE] Apache Flink CDC Release 3.4.0, release candidate #2

2025-04-28 Thread Hongshun Wang
Hi Yanquan, 1. Xiqian has found that significant performance regressions in FLINK-37104[1]. It's better to block until this is fixed. 2. FLINK-37738[2] n your release note is still not ready, maybe we can modify it to 3.4.1 or until it is merged. [1] https://issues.apache.org/jira/browse/FLINK-37

[jira] [Created] (FLINK-37738) [cdc-connector][postgres] Support read changelog as append only mode

2025-04-27 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37738: - Summary: [cdc-connector][postgres] Support read changelog as append only mode Key: FLINK-37738 URL: https://issues.apache.org/jira/browse/FLINK-37738 Project

[jira] [Created] (FLINK-37696) replace postgres image to official pg 14 and adjust test to use pgoutput plugin

2025-04-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37696: - Summary: replace postgres image to official pg 14 and adjust test to use pgoutput plugin Key: FLINK-37696 URL: https://issues.apache.org/jira/browse/FLINK-37696

Re: [ANNOUNCE] New Apache Flink Committer - Yanquan Lv

2025-04-16 Thread Hongshun Wang
Congrats Yanquan ! Best, Hongshun On Wed, Apr 16, 2025 at 9:04 PM gongzhongqiang wrote: > Congrats Yanquan Lv! Well deserved. > > Best, > Zhongqiang Gong > > Leonard Xu 于2025年4月16日周三 15:24写道: > > > Hi everyone, > > > > On behalf of the PMC, I'm happy to let you know that Yanquan Lv has > becom

Re: [ANNOUNCE] New Apache Flink Committer - Xiqian Yu

2025-04-16 Thread Hongshun Wang
Congratulations! Best Hongshun > 2025年4月16日 15:25,Leonard Xu 写道: > > becoming

Re: [VOTE] Release flink-connector-elasticsearch v3.1.0, release candidate #1

2025-04-13 Thread Hongshun Wang
+1 (non-binding) - verified JIRA release notes - Verify hashes and verify signatures - verified release tag, CI, and staging repo - reviewed web PR Best Hongshun On Wed, Apr 9, 2025 at 10:49 PM Ferenc Csaky wrote: > +1 (non-binding) > > - verified checksum and signature > - checked no binarie

[jira] [Created] (FLINK-37641) Fix: SchemaEvolveE2eITCase.testLenientSchemaEvolution timeout

2025-04-09 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37641: - Summary: Fix: SchemaEvolveE2eITCase.testLenientSchemaEvolution timeout Key: FLINK-37641 URL: https://issues.apache.org/jira/browse/FLINK-37641 Project: Flink

Re: How to add flink bot in flink subprojects(such as flink-connector-kafka)

2025-04-08 Thread Hongshun Wang
Hi Robert, It seems that flinkbot was added by you, would you like to give some introduction. Best Hongshun On Wed, Apr 9, 2025 at 1:51 PM Hongshun Wang wrote: > Dear Devs, > > Currently, flinkbot[1] is very helpful to rerun failed tests. Anyone know > how to add this to flink subp

How to add flink bot in flink subprojects(such as flink-connector-kafka)

2025-04-08 Thread Hongshun Wang
Dear Devs, Currently, flinkbot[1] is very helpful to rerun failed tests. Anyone know how to add this to flink subprojects(such as flink-connector-kafka, flink cdc)? It's inconvenient for developers who are not a committer. After each modification or unstable fail test, we have to ping a committer

Re: [VOTE] Release flink-connector-elasticsearch v4.0.0, release candidate #1

2025-04-08 Thread Hongshun Wang
Hi Weijie Should we update NOTICE[1] to 2025 before release? [1] https://github.com/apache/flink-connector-elasticsearch/blob/main/NOTICE Best Hongshun On Mon, Apr 7, 2025 at 8:06 PM Zakelly Lan wrote: > +1 (binding) > > I have verified: > > - Checksum and signature > - There are no binaries

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

2025-04-08 Thread Hongshun Wang
Hi Heise Should we update NOTICE[1] to 2025 before release? [1] https://github.com/apache/flink-connector-kafka/blob/main/NOTICE Best Hongshun

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-07 Thread Hongshun Wang
> prefix. But that is available earliest in Kafka 4.1, so we need the current > approach anyhow. > > Note that the FLIP is already approved, so I can't change anything on it > anymore. Both 1. and 3. can be discussed on the pull request however [1]. > 2. is clearly out of scop

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-06 Thread Hongshun Wang
> > > > > From: Arvid Heise > > Date: Wednesday, 26 March 2025 at 06:49 > > To: dev@flink.apache.org > > Subject: [EXTERNAL] Re: [DISCUSS] FLIP-511: Support transaction id > pooling in Kafka connector > > Hi Hongshun, > > > > this thread is abou

Re: [DISCUSS] FLIP-515: Dynamic Kafka Sink

2025-04-02 Thread Hongshun Wang
is updated with MetadataUpdateEvent. How is it going in Sink? I really hope you can draw a diagram like what FLIP-246 does. Best hongshun On Thu, Apr 3, 2025 at 1:54 PM Hongshun Wang wrote: > Hi Matyas, > > Sorry for the late reply. Dynamic Kafka Source has been introduced for a >

Re: [DISCUSS] FLIP-515: Dynamic Kafka Sink

2025-04-02 Thread Hongshun Wang
Hi Matyas, Sorry for the late reply. Dynamic Kafka Source has been introduced for a long time, and it is a little confusing for developers like me that KafkaDynamicSource and DynamicKafkaSource have the same kafka connector repo for different things. Now DynamicKafkaSink will be introduced again.

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-03-24 Thread Hongshun Wang
-34554 2. Too much unclosed producer ( if commit is lag too much from pre-commit: https://issues.apache.org/jira/browse/FLINK-36569 3. maybe Invalid state: https://issues.apache.org/jira/browse/FLINK-37356 I do hope this flip can show how to solve this problem. Best Hongshun Wang On Fri, Feb 28

[jira] [Created] (FLINK-37529) Add flink-cdc-commo to the package to all the incremental source framework

2025-03-21 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37529: - Summary: Add flink-cdc-commo to the package to all the incremental source framework Key: FLINK-37529 URL: https://issues.apache.org/jira/browse/FLINK-37529 Project

[jira] [Created] (FLINK-37509) Increment scan source no need to buffer data when enable skipping backfill.

2025-03-18 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37509: - Summary: Increment scan source no need to buffer data when enable skipping backfill. Key: FLINK-37509 URL: https://issues.apache.org/jira/browse/FLINK-37509

[jira] [Created] (FLINK-37508) Postgres CDC Jdbc query should use debezium.snapshot.fetch.size rather than debezium.query.fetch.size.

2025-03-18 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37508: - Summary: Postgres CDC Jdbc query should use debezium.snapshot.fetch.size rather than debezium.query.fetch.size. Key: FLINK-37508 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-37484) unbounded-chunk-first.enabled no use in unevently chunk split

2025-03-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37484: - Summary: unbounded-chunk-first.enabled no use in unevently chunk split Key: FLINK-37484 URL: https://issues.apache.org/jira/browse/FLINK-37484 Project: Flink

[jira] [Created] (FLINK-37396) Kafka connector python test fail

2025-02-27 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37396: - Summary: Kafka connector python test fail Key: FLINK-37396 URL: https://issues.apache.org/jira/browse/FLINK-37396 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-37380) Change TransactionalIdPrefix to a required option if Exactly once in Kafka Connector

2025-02-24 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37380: - Summary: Change TransactionalIdPrefix to a required option if Exactly once in Kafka Connector Key: FLINK-37380 URL: https://issues.apache.org/jira/browse/FLINK-37380

Re: [ANNOUNCE] New Apache Flink Committer - Xuyang

2025-02-19 Thread Hongshun Wang
Congratulations, Xuyang! You deserve it Best, Hongshun On Thu, Feb 20, 2025 at 11:26 AM Shawn Huang wrote: > Congratulations ! > > Best, > Shawn Huang > > > Yanquan Lv 于2025年2月20日周四 11:05写道: > > > Congratulations, Xuyang! > > > > Best, Yanquan. > > > > > 2025年2月20日 10:24,Zakelly Lan 写道: > > >

[jira] [Created] (FLINK-37356) Recycle use of kafka producer(which commit error) will send data without AddPartitionsToTxnRequest

2025-02-19 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-37356: - Summary: Recycle use of kafka producer(which commit error) will send data without AddPartitionsToTxnRequest Key: FLINK-37356 URL: https://issues.apache.org/jira/browse/FLINK

Re: [Discuss] Kafka Enumerator assigns splits based on the current assignment situation.

2024-12-30 Thread Hongshun Wang
ke just a round-robin > assignment for all topic/partitions instead of hash based topic assignment > + round-robin partition? > > I am sorry if the discussion is getting more detailed, happy to move it to > a FLIP if the community agrees as well. > > Best Regards > Ahmed Hamdy &

Re: [Discuss] Kafka Enumerator assigns splits based on the current assignment situation.

2024-12-29 Thread Hongshun Wang
based on the current circumstances. Best, Hongshun On Mon, Dec 30, 2024 at 10:03 AM Hongshun Wang wrote: > Hi Ahmed, > > Thanks for your question. > > Do you intend to keep the current assignment in the enumerator > state or are you only going to use per session assignment statu

Re: [Discuss] Kafka Enumerator assigns splits based on the current assignment situation.

2024-12-29 Thread Hongshun Wang
lism change? > > > 1- https://issues.apache.org/jira/browse/FLINK-31762 > Best Regards > Ahmed Hamdy > > > On Fri, 27 Dec 2024 at 06:38, Hongshun Wang > wrote: > > > Hi devs, > > > > I'd like to push a new FLIP xxx: Kafka Enumerator assigns spl

Re: [DISCUSS] FLIP-467: Introduce Generalized Watermarks

2024-12-29 Thread Hongshun Wang
Hi Xu Hang, Sorry, it's late to join this. Thanks for your job, it make sense to me. I have several questions: > With this abstraction, the original event-time watermark can be seen as a built-in use case of it. I wonder whether to replace the older event-time watermark strategy in this FLIP or

[Discuss] Kafka Enumerator assigns splits based on the current assignment situation.

2024-12-26 Thread Hongshun Wang
Hi devs, I'd like to push a new FLIP xxx: Kafka Enumerator assigns splits based on the current assignment situation. Currently, when the Kafka enumerator finds a new split, it will choose a reader using the following algorithm: ```java static int getSplitOwner(TopicPartition tp, int numReaders)

[jira] [Created] (FLINK-36844) Postgres CDC document add mapping from postgres timestamp_tz to flink timestamp ltz

2024-12-04 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36844: - Summary: Postgres CDC document add mapping from postgres timestamp_tz to flink timestamp ltz Key: FLINK-36844 URL: https://issues.apache.org/jira/browse/FLINK-36844

Re: FLIP-484: Add custom metric variables to operators

2024-12-02 Thread Hongshun Wang
Hi Piotr, thanks for the proposal. > it would be helpful to group metrics from each source/sink instance together, Could you please give a more detailed example how to do it? Currently, the example only shows source. Thanks, Hongshun On Thu, Nov 14, 2024 at 8:36 PM Roman Khachatryan wrote:

[jira] [Created] (FLINK-36618) Improve PostgresDialect.discoverDataCollections to reduce the start time of Postgres CDC

2024-10-28 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36618: - Summary: Improve PostgresDialect.discoverDataCollections to reduce the start time of Postgres CDC Key: FLINK-36618 URL: https://issues.apache.org/jira/browse/FLINK-36618

[jira] [Created] (FLINK-36609) Protobuf Format not support proto3

2024-10-27 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36609: - Summary: Protobuf Format not support proto3 Key: FLINK-36609 URL: https://issues.apache.org/jira/browse/FLINK-36609 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-480: Support to deploy script in application mode

2024-10-27 Thread Hongshun Wang
Hi, ShengKai Thanks for pushing for this meaningful job. I have some problems: Firstly, the outline design show the overall design in k8s. I wonder what is the scope of this FLIP, only aim for k8s, not including yarn? Secondly, > Flink offers great flexibility by permitting users to pass custom

[jira] [Created] (FLINK-36174) CDC yaml without pipeline will throw NullPointException

2024-08-29 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36174: - Summary: CDC yaml without pipeline will throw NullPointException Key: FLINK-36174 URL: https://issues.apache.org/jira/browse/FLINK-36174 Project: Flink

[jira] [Created] (FLINK-36163) Kafka cdc pipeline source supports protobuf format.

2024-08-27 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36163: - Summary: Kafka cdc pipeline source supports protobuf format. Key: FLINK-36163 URL: https://issues.apache.org/jira/browse/FLINK-36163 Project: Flink Issue

Re: Flink CDC insert-only changelog mode

2024-08-25 Thread Hongshun Wang
:45 AM Daniel Henneberger wrote: > Unfortunately no, this would not be helpful. The changelog type needs to be > insert-only to invoke the correct streaming planner rules. > > On Wed, Aug 21, 2024 at 9:01 PM Hongshun Wang > wrote: > > > Hi Daniel, > > > > >

[jira] [Created] (FLINK-36150) tables.exclude is not valid if scan.binlog.newly-added-table.enabled is true.

2024-08-25 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36150: - Summary: tables.exclude is not valid if scan.binlog.newly-added-table.enabled is true. Key: FLINK-36150 URL: https://issues.apache.org/jira/browse/FLINK-36150

Re: Flink CDC insert-only changelog mode

2024-08-21 Thread Hongshun Wang
Hi Daniel, > > There's no way to convert a different changelog stream to an insertonly > stream so I'm pushing this upstream to the connector. I wonder whether you just want the insert type changelog and ignore other kinds of changelog, or just want to read all kinds of changelog as insert in fli

[jira] [Created] (FLINK-36094) CDC SchemaRegistryRequestHandler should throw exception which is not SchemaEvolveException

2024-08-19 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36094: - Summary: CDC SchemaRegistryRequestHandler should throw exception which is not SchemaEvolveException Key: FLINK-36094 URL: https://issues.apache.org/jira/browse/FLINK-36094

[jira] [Created] (FLINK-36076) Hotfix: Set isSchemaChangeApplying as isSchemaChangeApplying for thread safe.

2024-08-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36076: - Summary: Hotfix: Set isSchemaChangeApplying as isSchemaChangeApplying for thread safe. Key: FLINK-36076 URL: https://issues.apache.org/jira/browse/FLINK-36076

[jira] [Created] (FLINK-36036) CDC remove unused dependency jar from calcite-core

2024-08-12 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36036: - Summary: CDC remove unused dependency jar from calcite-core Key: FLINK-36036 URL: https://issues.apache.org/jira/browse/FLINK-36036 Project: Flink Issue

[jira] [Created] (FLINK-36023) Flink K8S Native Application Mode add wrong jar url.

2024-08-09 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36023: - Summary: Flink K8S Native Application Mode add wrong jar url. Key: FLINK-36023 URL: https://issues.apache.org/jira/browse/FLINK-36023 Project: Flink Issue

[jira] [Created] (FLINK-36007) cdc load and register factory in once search

2024-08-08 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-36007: - Summary: cdc load and register factory in once search Key: FLINK-36007 URL: https://issues.apache.org/jira/browse/FLINK-36007 Project: Flink Issue Type

[jira] [Created] (FLINK-35994) CDC pipeline use SavepointRestoreSettings in flink config flile as a fallback strategy.

2024-08-07 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35994: - Summary: CDC pipeline use SavepointRestoreSettings in flink config flile as a fallback strategy. Key: FLINK-35994 URL: https://issues.apache.org/jira/browse/FLINK-35994

[jira] [Created] (FLINK-35968) Remove flink-cdc-runtime depedency from connectors

2024-08-02 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35968: - Summary: Remove flink-cdc-runtime depedency from connectors Key: FLINK-35968 URL: https://issues.apache.org/jira/browse/FLINK-35968 Project: Flink Issue

[jira] [Created] (FLINK-35859) [flink-cdc] Fix: The assigner is not ready to offer finished split information, this should not be called

2024-07-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35859: - Summary: [flink-cdc] Fix: The assigner is not ready to offer finished split information, this should not be called Key: FLINK-35859 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-35849) [flink-cdc] Use expose_snapshot to read snapshot of postgres cdc connector.

2024-07-15 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35849: - Summary: [flink-cdc] Use expose_snapshot to read snapshot of postgres cdc connector. Key: FLINK-35849 URL: https://issues.apache.org/jira/browse/FLINK-35849

[jira] [Created] (FLINK-35760) [docs] Add scan.newly-added-table.enabled to docs.

2024-07-04 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35760: - Summary: [docs] Add scan.newly-added-table.enabled to docs. Key: FLINK-35760 URL: https://issues.apache.org/jira/browse/FLINK-35760 Project: Flink Issue

[jira] [Created] (FLINK-35759) [docs] Add 'scan.incremental.snapshot.backfill.skip' to docs.

2024-07-04 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35759: - Summary: [docs] Add 'scan.incremental.snapshot.backfill.skip' to docs. Key: FLINK-35759 URL: https://issues.apache.org/jira/browse/FLINK-35759 Proj

[jira] [Created] (FLINK-35715) Mysql Source support schema cache to deserialize record

2024-06-27 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35715: - Summary: Mysql Source support schema cache to deserialize record Key: FLINK-35715 URL: https://issues.apache.org/jira/browse/FLINK-35715 Project: Flink

[jira] [Created] (FLINK-35693) Change variable specificOffsetFile, specificOffsetPos and startupTimestampMillis to Offset in StartupOptions

2024-06-25 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35693: - Summary: Change variable specificOffsetFile, specificOffsetPos and startupTimestampMillis to Offset in StartupOptions Key: FLINK-35693 URL: https://issues.apache.org/jira

Re: [ANNOUNCE] New Apache Flink Committer - Hang Ruan

2024-06-17 Thread Hongshun Wang
Congratulations Hang! Best, Hongshun On Mon, Jun 17, 2024 at 5:49 PM Ron Liu wrote: > Congratulations, Hang! > > Best, > Ron > > Geng Biao 于2024年6月17日周一 12:35写道: > > > Congrats, Hang! > > Best, > > Biao Geng > > > > 发送自 Outlook for iOS > > ___

Re: [ANNOUNCE] New Apache Flink Committer - Zhongqiang Gong

2024-06-17 Thread Hongshun Wang
Congrats Zhongqiang ! Best, Hongshun On Mon, Jun 17, 2024 at 12:35 PM Geng Biao wrote: > Congratulations, Zhongqiang! > Best, > Biao Geng > > 发送自 Outlook for iOS > > 发件人: Zakelly Lan > 发送时间: Monday, June 17, 2024 12:11:47 PM > 收件人: dev@fl

[jira] [Created] (FLINK-35567) CDC BinaryWriter cast NullableSerializerWrapper error

2024-06-11 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35567: - Summary: CDC BinaryWriter cast NullableSerializerWrapper error Key: FLINK-35567 URL: https://issues.apache.org/jira/browse/FLINK-35567 Project: Flink

[jira] [Created] (FLINK-35524) Clear connections pools when reader exist.

2024-06-05 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35524: - Summary: Clear connections pools when reader exist. Key: FLINK-35524 URL: https://issues.apache.org/jira/browse/FLINK-35524 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-35387) PG CDC source support heart beat

2024-05-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35387: - Summary: PG CDC source support heart beat Key: FLINK-35387 URL: https://issues.apache.org/jira/browse/FLINK-35387 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-35385) upgrader flink dependency version to 1.19

2024-05-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35385: - Summary: upgrader flink dependency version to 1.19 Key: FLINK-35385 URL: https://issues.apache.org/jira/browse/FLINK-35385 Project: Flink Issue Type

[jira] [Created] (FLINK-35349) Use connection in openJdbcConnection of SqlServerDialect/Db2Dialect/OracleDialect

2024-05-13 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35349: - Summary: Use connection in openJdbcConnection of SqlServerDialect/Db2Dialect/OracleDialect Key: FLINK-35349 URL: https://issues.apache.org/jira/browse/FLINK-35349

[jira] [Created] (FLINK-35344) Move same code from multiple subclasses to JdbcSourceChunkSplitter

2024-05-13 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35344: - Summary: Move same code from multiple subclasses to JdbcSourceChunkSplitter Key: FLINK-35344 URL: https://issues.apache.org/jira/browse/FLINK-35344 Project: Flink

Re: [DISCUSS] Connector releases for Flink 1.19

2024-05-12 Thread Hongshun Wang
Hello Danny, Thanks for pushing this forward. I am available to assist with the CDC connector[1]. [1] https://github.com/apache/flink-cdc Best Hongshun On Sun, May 12, 2024 at 8:48 PM Sergey Nuyanzin wrote: > I'm in a process of preparation of RC for OpenSearch connector > > however it seems

[jira] [Created] (FLINK-35327) SQL Explain show push down condition

2024-05-09 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35327: - Summary: SQL Explain show push down condition Key: FLINK-35327 URL: https://issues.apache.org/jira/browse/FLINK-35327 Project: Flink Issue Type: Bug

Re: [DISCUSS] Flink CDC 3.2 Release Planning

2024-05-08 Thread Hongshun Wang
Thanks Qinsheng for driving, +1 from my side. Besi, Hongshun On Wed, May 8, 2024 at 11:41 PM Leonard Xu wrote: > +1 for the proposal code freeze date and RM candidate. > > Best, > Leonard > > > 2024年5月8日 下午10:27,gongzhongqiang 写道: > > > > Hi Qingsheng > > > > Thank you for driving the release.

Re: [DISCUSS] FLIP-453: Promote Unified Sink API V2 to Public and Deprecate SinkFunction

2024-05-07 Thread Hongshun Wang
Hi Martijn, Thanks for the proposal +1 from me.Some sinks still use sinkfunction; it's time to take a step forward. Best, Hongshun On Mon, May 6, 2024 at 5:44 PM Leonard Xu wrote: > +1 from my side, thanks Martijn for the effort. > > Best, > Leonard > > > 2024年5月4日 下午7:41,Ahmed Hamdy 写道: > > >

[jira] [Created] (FLINK-35281) FlinkEnvironmentUtils#addJar add each jar only once

2024-05-01 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35281: - Summary: FlinkEnvironmentUtils#addJar add each jar only once Key: FLINK-35281 URL: https://issues.apache.org/jira/browse/FLINK-35281 Project: Flink Issue

[jira] [Created] (FLINK-35234) Fix NullPointerException of org.apache.flink.cdc.common.configuration.ConfigurationUtils#convertToString

2024-04-24 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35234: - Summary: Fix NullPointerException of org.apache.flink.cdc.common.configuration.ConfigurationUtils#convertToString Key: FLINK-35234 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-35149) Fix DataSinkTranslator#sinkTo ignoring pre-write topology if not TwoPhaseCommittingSink

2024-04-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35149: - Summary: Fix DataSinkTranslator#sinkTo ignoring pre-write topology if not TwoPhaseCommittingSink Key: FLINK-35149 URL: https://issues.apache.org/jira/browse/FLINK-35149

[jira] [Created] (FLINK-35143) Expose newly added tables capture in mysql pipeline connector

2024-04-17 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35143: - Summary: Expose newly added tables capture in mysql pipeline connector Key: FLINK-35143 URL: https://issues.apache.org/jira/browse/FLINK-35143 Project: Flink

[jira] [Created] (FLINK-35129) Postgres source commits the offset after every multiple checkpoint cycles.

2024-04-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35129: - Summary: Postgres source commits the offset after every multiple checkpoint cycles. Key: FLINK-35129 URL: https://issues.apache.org/jira/browse/FLINK-35129 Project

[jira] [Created] (FLINK-35128) Re-calculate the starting binlog offset after the new table added

2024-04-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35128: - Summary: Re-calculate the starting binlog offset after the new table added Key: FLINK-35128 URL: https://issues.apache.org/jira/browse/FLINK-35128 Project: Flink

[jira] [Created] (FLINK-35121) CDC pipeline connector should verify requiredOptions and optionalOptions

2024-04-16 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35121: - Summary: CDC pipeline connector should verify requiredOptions and optionalOptions Key: FLINK-35121 URL: https://issues.apache.org/jira/browse/FLINK-35121 Project

  1   2   >