[jira] [Created] (FLINK-34016) Janino compile failed when watermark with column by udf

2024-01-07 Thread Jude Zhu (Jira)
Jude Zhu created FLINK-34016: Summary: Janino compile failed when watermark with column by udf Key: FLINK-34016 URL: https://issues.apache.org/jira/browse/FLINK-34016 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-403: High Availability Services for OLAP Scenarios

2024-01-07 Thread Rui Fan
Thanks to Yangze driving this proposal! Overall looks good to me! This proposal is useful for the performance when the job doesn't need the failover. I have some minor questions: 1. How does it work with FLIP-383[1]? This FLIP introduces a high-availability.enable-job-recovery, and FLIP-383

Re: [Discuss] FLIP-407: Improve Flink Client performance in interactive scenarios

2024-01-07 Thread Rui Fan
Only one strategy is fine to me. When the multiplier is set to 1, the exponential-delay will become fixed-delay. So fixed-delay may not be needed. Best, Rui On Mon, Jan 8, 2024 at 2:17 PM Yong Fang wrote: > I agree with @Rui that the current configuration for Flink Client is a > little

Re: FLIP-413: Enable unaligned checkpoints by default

2024-01-07 Thread Rui Fan
Thanks to Piotr driving this proposal! Enabling unaligned checkpoint with aligned checkpoints timeout is fine for me. I'm not sure if aligned checkpoints timeout =5s is too aggressive. If the unaligned checkpoint is enabled by default for all jobs, I recommend that the aligned checkpoints timeout

[jira] [Created] (FLINK-34015) execution.savepoint.ignore-unclaimed-state is invalid when passing this parameter by dynamic properties

2024-01-07 Thread Renxiang Zhou (Jira)
Renxiang Zhou created FLINK-34015: - Summary: execution.savepoint.ignore-unclaimed-state is invalid when passing this parameter by dynamic properties Key: FLINK-34015 URL:

Re: [Discuss] FLIP-407: Improve Flink Client performance in interactive scenarios

2024-01-07 Thread Yong Fang
I agree with @Rui that the current configuration for Flink Client is a little complex. Can we just provide one strategy with less configuration items for all scenarios? Best, Fang Yong On Mon, Jan 8, 2024 at 11:19 AM Rui Fan <1996fan...@gmail.com> wrote: > Thanks xiangyu for driving this

Re: [DISCUSS] FLIP-403: High Availability Services for OLAP Scenarios

2024-01-07 Thread Yangze Guo
Thanks for your comment, Yong. Here are my thoughts on the splitting of HighAvailableServices: Firstly, I would treat this separation as a result of technical debt and a side effect of the FLIP. In order to achieve a cleaner interface hierarchy for High Availability before Flink 2.0, the design

Re: [VOTE] FLIP-405: Migrate string configuration key to ConfigOption

2024-01-07 Thread Hang Ruan
+1(non-binding) Best, Hang Rui Fan <1996fan...@gmail.com> 于2024年1月8日周一 13:04写道: > +1(binding) > > Best, > Rui > > On Mon, Jan 8, 2024 at 1:00 PM Xuannan Su wrote: > > > Hi everyone, > > > > Thanks for all the feedback about the FLIP-405: Migrate string > > configuration key to ConfigOption [1]

Re:Re:Re: Re: [DISCUSS] FLIP-392: Deprecate the Legacy Group Window Aggregation

2024-01-07 Thread Xuyang
Hi, all. Considering what Martijn suggested, as a big break, we can directly remove the old group window agg operator in 2.0 without rewriting it to the new window agg operator. At the same time, every subtask in this FLIP to align the new and old window agg operators is almost covered by

Re: [VOTE] FLIP-405: Migrate string configuration key to ConfigOption

2024-01-07 Thread Rui Fan
+1(binding) Best, Rui On Mon, Jan 8, 2024 at 1:00 PM Xuannan Su wrote: > Hi everyone, > > Thanks for all the feedback about the FLIP-405: Migrate string > configuration key to ConfigOption [1] [2]. > > I'd like to start a vote for it. The vote will be open for at least 72 > hours(excluding

[VOTE] FLIP-405: Migrate string configuration key to ConfigOption

2024-01-07 Thread Xuannan Su
Hi everyone, Thanks for all the feedback about the FLIP-405: Migrate string configuration key to ConfigOption [1] [2]. I'd like to start a vote for it. The vote will be open for at least 72 hours(excluding weekends,until Jan 11, 12:00AM GMT) unless there is an objection or an insufficient number

Re: FLIP-413: Enable unaligned checkpoints by default

2024-01-07 Thread Zhanghao Chen
Hi Piotr, As a platform administer who runs kilos of Flink jobs, I'd be against the idea to enable unaligned cp by default for our jobs. It may help a significant portion of the users, but the subtle issues around unaligned CP for a few jobs will probably raise a lot more on-calls and

Re: [DISCUSS] FLIP-403: High Availability Services for OLAP Scenarios

2024-01-07 Thread Yong Fang
Thanks Yangze for starting this discussion. I have one comment: why do we need to abstract two services as `LeaderServices` and `PersistenceServices`? >From the content, the purpose of this FLIP is to make job failover more lightweight, so it would be more appropriate to abstract two services as

[jira] [Created] (FLINK-34014) Jdbc connector can avoid send empty insert to database when there's no buffer data

2024-01-07 Thread luoyuxia (Jira)
luoyuxia created FLINK-34014: Summary: Jdbc connector can avoid send empty insert to database when there's no buffer data Key: FLINK-34014 URL: https://issues.apache.org/jira/browse/FLINK-34014 Project:

Re: [DISCUSS] FLIP-398: Improve Serialization Configuration And Usage In Flink

2024-01-07 Thread Yong Fang
Hi Ken, I think the main reason is that currently Kryo is the only generic serializer in Flink. I'm looking forward to your FLIP of Fury, and we can continue to discuss this issue there. If there are no other questions, I will close the voting for this FLIP. Thank you again. Best, Fang Yong On

[DISCUSS] FLIP-414: Support Retry Mechanism in RocksDBStateDataTransfer

2024-01-07 Thread xiangyu feng
Hi devs, I'm opening this thread to discuss FLIP-414: Support Retry Mechanism in RocksDBStateDataTransfer[1]. Currently, there is no retry mechanism for downloading and uploading RocksDB state files. Any jittering of remote filesystem might lead to a checkpoint failure. By supporting retry

Re: [Discuss] FLIP-407: Improve Flink Client performance in interactive scenarios

2024-01-07 Thread Rui Fan
Thanks xiangyu for driving this proposal! And sorry for the late reply. Overall looks good to me, I only have some minor questions: 1. Do we need to introduce 3 collect strategies in the first version? Large and comprehensive configuration items will bring additional learning costs and usage

Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2024-01-07 Thread Yun Tang
Hi Zakelly, Thanks for driving this topic. I have two concerns here: 1. We shall not describe the configuration with its implementation for ​'execution.checkpointing.local-copy.*' options, for hashmap state-backend, it would write two streams and for Rocksdb state-backend, it would use

Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2024-01-07 Thread Xuannan Su
Hi all, Thanks for the discussion. I think all the comments and questions have been addressed. I will open the voting thread today. Best, Xuannan On Tue, Jan 2, 2024 at 11:59 AM Xuannan Su wrote: > > Hi all, > > Thank you for all your comments! The FLIP has been updated > accordingly. Please

Re: [DISCUSS] FLIP-329: Add operator attribute to specify support for object-reuse

2024-01-07 Thread Xuannan Su
Hi Liu, The voting thread has been open for a long time. We may want to start a new voting thread. WDYT? Best, Xuannan On Sat, Jan 6, 2024 at 1:51 AM Lu Niu wrote: > > Thank you Dong and Xuannan! > > Yes. We can take on this task. Any help during bootstrapping would be greatly > appreciated!

[jira] [Created] (FLINK-34013) ProfilingServiceTest.testRollingDeletion is unstable on AZP

2024-01-07 Thread Sergey Nuyanzin (Jira)
Sergey Nuyanzin created FLINK-34013: --- Summary: ProfilingServiceTest.testRollingDeletion is unstable on AZP Key: FLINK-34013 URL: https://issues.apache.org/jira/browse/FLINK-34013 Project: Flink

[jira] [Created] (FLINK-34012) Flink python fails with can't read file '/__w/2/s/flink-python/dev/.conda/lib/python3.10/site-packages//google

2024-01-07 Thread Sergey Nuyanzin (Jira)
Sergey Nuyanzin created FLINK-34012: --- Summary: Flink python fails with can't read file '/__w/2/s/flink-python/dev/.conda/lib/python3.10/site-packages//google Key: FLINK-34012 URL:

[jira] [Created] (FLINK-34011) EmbeddedThreadDependencyTests.test_add_python_file fails with This function should not be called!

2024-01-07 Thread Sergey Nuyanzin (Jira)
Sergey Nuyanzin created FLINK-34011: --- Summary: EmbeddedThreadDependencyTests.test_add_python_file fails with This function should not be called! Key: FLINK-34011 URL:

[jira] [Created] (FLINK-34010) KafkaTableITCase.testPerPartitionWatermarkWithIdleSource is unstable on AZP

2024-01-07 Thread Sergey Nuyanzin (Jira)
Sergey Nuyanzin created FLINK-34010: --- Summary: KafkaTableITCase.testPerPartitionWatermarkWithIdleSource is unstable on AZP Key: FLINK-34010 URL: https://issues.apache.org/jira/browse/FLINK-34010

Re: [DISCUSS] FLIP-413: Enable unaligned checkpoints by default

2024-01-07 Thread Ken Krugler
In line with what David said, after having to explain the (often subtle) issues around unaligned checkpoints and upgrades while teaching Flink, I would also be concerned about enabling it by default. Would it be better to provide more automatic detection of situations where unaligned

Re: [DISCUSS] FLIP-413: Enable unaligned checkpoints by default

2024-01-07 Thread David Anderson
Piotr, I think the situation is more nuanced than what you've described. One concern I have is that unaligned checkpoints are somewhat less flexible in terms of which operational tasks can be safely performed with them -- i.e., if you look at the table in the docs [1], aligned checkpoints support

Re: [VOTE] FLIP-387: Support named parameters for functions and call procedures

2024-01-07 Thread Feng Jin
Hi Alexey Thank you for the reminder, the link has been updated. Best, Feng Jin On Sat, Jan 6, 2024 at 12:55 AM Alexey Leonov-Vendrovskiy < vendrov...@gmail.com> wrote: > Thanks for starting the vote! > Do you mind adding a link from the FLIP to this thread? > > Thanks, > Alexey > > On Thu,

[jira] [Created] (FLINK-34009) Apache flink: Checkpoint restoration issue on Application Mode of deployment

2024-01-07 Thread Vijay (Jira)
Vijay created FLINK-34009: - Summary: Apache flink: Checkpoint restoration issue on Application Mode of deployment Key: FLINK-34009 URL: https://issues.apache.org/jira/browse/FLINK-34009 Project: Flink

RE: [VOTE] Release flink-connector-jdbc, release candidate #1

2024-01-07 Thread David Radley
Hi , I am working on FLINK-33365. I am making good progress; thanks Sergey for your fabulous feedback. A lot of the query cases are now working with the latest fix but not all. I think it is pragmatic to revert the lookup join predicate pushdown support, so we can release a functional JDBC