Re: [DISCUSS] FLIP-XXX Support currentFetchEventTimeLag and processingLag metrics

2024-04-28 Thread jialiang tan
Thanks Qingsheng for the patient guidance! And sorry to everyone for wasting your time. I suddenly realised that my implementation is wrong. In my implementation the FetchTime and EventTime do not come from the same record and I am making this stupid mistake :( . I need some time to think about

Re: [DISCUSS] FLIP-XXX Support currentFetchEventTimeLag and processingLag metrics

2024-04-28 Thread Qingsheng Ren
Hi Jialiang, Thanks for the FLIP! Here're some thoughts of mine. - For currentFetchEventTimeLag: The problem of currentFetchEventTimeLag is: FetchTime is determined in SplitReader driven by SplitFetcher thread, while EventTime is calculated at the output of SourceOperator driven by task's main t

Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-04-28 Thread Ron Liu
Hi, Lorenzo > I have a question there: how can the gateway update the refreshHandler in the Catalog before getting it from the scheduler? The refreshHandler in CatalogMateriazedTable is null before getting it from the scheduler, you can look at the CatalogMaterializedTable.Builder[1] for more det

[jira] [Created] (FLINK-35261) Flink CDC pipeline transform doesn't support decimal comparison

2024-04-28 Thread yux (Jira)
yux created FLINK-35261: --- Summary: Flink CDC pipeline transform doesn't support decimal comparison Key: FLINK-35261 URL: https://issues.apache.org/jira/browse/FLINK-35261 Project: Flink Issue Type: Im

[jira] [Created] (FLINK-35260) Translate "Watermark alignment "page into Chinese

2024-04-28 Thread hongxu han (Jira)
hongxu han created FLINK-35260: -- Summary: Translate "Watermark alignment "page into Chinese Key: FLINK-35260 URL: https://issues.apache.org/jira/browse/FLINK-35260 Project: Flink Issue Type: Imp

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

2024-04-28 Thread Aleksandr Pilipenko
+1 (non-binding) - Validated checksum - Verified signature - Checked that no binaries exist in the source archive - Build source - Verified web PR Thanks, Aleksandr On Sun, 28 Apr 2024 at 11:35, Hang Ruan wrote: > +1 (non-binding) > > - Validated checksum hash > - Verified signature > - Verifi

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

2024-04-28 Thread Aleksandr Pilipenko
+1 (non-binding) - Verified checksums - Verified signatures - Checked that no binaries exist in the source archive - Reviewed Web PR - Built source Thanks, Aleksandr On Mon, 22 Apr 2024 at 09:31, Ahmed Hamdy wrote: > Thanks Danny, > +1 (non-binding) > > - Verified Checksums > - Verified Signat

Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-04-28 Thread Ron Liu
Hi, Shengkai Thanks for your feedback and suggestion, it looks very useful for this proposal, regarding your question I made the following optimization: > *WorkflowScheduler* > 1. How to get the exception details if `modifyRefreshWorkflow` fails? > 2. Could you give us an example about how to con

[jira] [Created] (FLINK-35259) FlinkCDC Pipeline transform can't deal timestamp field

2024-04-28 Thread Wenkai Qi (Jira)
Wenkai Qi created FLINK-35259: - Summary: FlinkCDC Pipeline transform can't deal timestamp field Key: FLINK-35259 URL: https://issues.apache.org/jira/browse/FLINK-35259 Project: Flink Issue Type:

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

2024-04-28 Thread Hang Ruan
+1 (non-binding) - Validated checksum hash - Verified signature - Verified that no binaries exist in the source archive - Build the source with Maven and jdk8 - Verified web PR - Check that the jar is built by jdk8 Best, Hang Ahmed Hamdy 于2024年4月24日周三 17:21写道: > Thanks Danny, > +1 (non-binding

[jira] [Created] (FLINK-35258) Broken links to Doris in Flink CDC Documentation

2024-04-28 Thread Qingsheng Ren (Jira)
Qingsheng Ren created FLINK-35258: - Summary: Broken links to Doris in Flink CDC Documentation Key: FLINK-35258 URL: https://issues.apache.org/jira/browse/FLINK-35258 Project: Flink Issue Typ

Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-04-28 Thread Shengkai Fang
Hi, Liu. Thanks for your proposal. I have some questions about the FLIP: *WorkflowScheduler* 1. How to get the exception details if `modifyRefreshWorkflow` fails? 2. Could you give us an example about how to configure the scheduler? *SQL Gateway* 1. SqlGatewayService requires Session as the in