Re: [VOTE] FLIP-187: Adaptive Batch Job Scheduler

2021-11-09 Thread Zhu Zhu
+1 (binding) Thanks, Zhu Jingsong Li 于2021年11月9日周二 下午4:01写道: > +1 (non-binding) > > This greatly enhances the ease of use of batch jobs. (Parallelism > setting is really a challenge) > > (non-binding: I'm not familiar with runtime, just a general understanding) > > Best, > Jingsong > > On Tue,

Re: [VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Jingsong Li
Hi Neng, Thanks for your voting and suggestions. Although we will not propose a public interface for LogStore, there is no doubt that it will be interfaced to avoid a hardcoded Kafka LogStore. Best, Jingsong On Wed, Nov 10, 2021 at 11:44 AM Kurt Young wrote: > > +1 (binding) > > Best, > Kurt

Re: [VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Kurt Young
+1 (binding) Best, Kurt On Wed, Nov 10, 2021 at 10:52 AM Jingsong Li wrote: > Hi everyone, > > Thanks for all the feedback so far. Based on the discussion[1] we seem > to have consensus, so I would like to start a vote on FLIP-188 for > which the FLIP has now also been updated[2]. > > The

Re: [VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Neng Lu
Hi, +1 for (non-binding) Also, I'm curious to see if we can avoid a hardcoded dependency on Kafka for the LogStore. This will make the solution more extensible. On 2021/11/10 02:51:55 Jingsong Li wrote: > Hi everyone, > > Thanks for all the feedback so far. Based on the discussion[1] we

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Jingsong Li
Hi all, I have started the voting thread [1]. Please cast your vote there or ask additional questions here. [1] https://lists.apache.org/thread/v3fzx0p6n2jogn86sptzr30kr3yw37sq Best, Jingsong On Mon, Nov 1, 2021 at 5:41 PM Jingsong Li wrote: > > Hi Till, > > Thanks for your suggestion. > > At

[VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Jingsong Li
Hi everyone, Thanks for all the feedback so far. Based on the discussion[1] we seem to have consensus, so I would like to start a vote on FLIP-188 for which the FLIP has now also been updated[2]. The vote will last for at least 72 hours (Nov 13th 3:00 GMT) unless there is an objection or

[jira] [Created] (FLINK-24852) Cleanup of Orphaned Incremental State Artifacts

2021-11-09 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-24852: Summary: Cleanup of Orphaned Incremental State Artifacts Key: FLINK-24852 URL: https://issues.apache.org/jira/browse/FLINK-24852 Project: Flink Issue Type:

[jira] [Created] (FLINK-24851) KafkaSourceBuilder: auto.offset.reset is ignored

2021-11-09 Thread Arseniy Tashoyan (Jira)
Arseniy Tashoyan created FLINK-24851: Summary: KafkaSourceBuilder: auto.offset.reset is ignored Key: FLINK-24851 URL: https://issues.apache.org/jira/browse/FLINK-24851 Project: Flink

[jira] [Created] (FLINK-24850) KafkaSinkBuilder: NullPointerException if bootstrap.servers are defined via properties

2021-11-09 Thread Arseniy Tashoyan (Jira)
Arseniy Tashoyan created FLINK-24850: Summary: KafkaSinkBuilder: NullPointerException if bootstrap.servers are defined via properties Key: FLINK-24850 URL: https://issues.apache.org/jira/browse/FLINK-24850

[jira] [Created] (FLINK-24849) Only return stacktrace if RestHandlerException has a cause

2021-11-09 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-24849: Summary: Only return stacktrace if RestHandlerException has a cause Key: FLINK-24849 URL: https://issues.apache.org/jira/browse/FLINK-24849 Project: Flink

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-11-09 Thread Roman Khachatryan
Hi everyone, Thanks for the proposal and the discussion, I have some remarks: (I'm not very familiar with the new Sink API but I thought about the same problem in context of the changelog state backend) 1. Merging artifacts from multiple checkpoints would apparently require multiple concurrent

[jira] [Created] (FLINK-24848) Improve error codes for savepoint handlers

2021-11-09 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-24848: Summary: Improve error codes for savepoint handlers Key: FLINK-24848 URL: https://issues.apache.org/jira/browse/FLINK-24848 Project: Flink Issue

[jira] [Created] (FLINK-24847) Decide the overflows behaviour

2021-11-09 Thread Francesco Guardiani (Jira)
Francesco Guardiani created FLINK-24847: --- Summary: Decide the overflows behaviour Key: FLINK-24847 URL: https://issues.apache.org/jira/browse/FLINK-24847 Project: Flink Issue Type:

[jira] [Created] (FLINK-24846) AsyncWaitOperator fails during stop-with-savepoint

2021-11-09 Thread Piotr Nowojski (Jira)
Piotr Nowojski created FLINK-24846: -- Summary: AsyncWaitOperator fails during stop-with-savepoint Key: FLINK-24846 URL: https://issues.apache.org/jira/browse/FLINK-24846 Project: Flink Issue

[jira] [Created] (FLINK-24845) Add allreduce utility function in FlinkML

2021-11-09 Thread Zhipeng Zhang (Jira)
Zhipeng Zhang created FLINK-24845: - Summary: Add allreduce utility function in FlinkML Key: FLINK-24845 URL: https://issues.apache.org/jira/browse/FLINK-24845 Project: Flink Issue Type: New

[jira] [Created] (FLINK-24844) CassandraConnectorITCase.testCassandraBatchPojoFormat fails on AZP

2021-11-09 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-24844: - Summary: CassandraConnectorITCase.testCassandraBatchPojoFormat fails on AZP Key: FLINK-24844 URL: https://issues.apache.org/jira/browse/FLINK-24844 Project: Flink

[DISCUSS] Should we go only with DateTimeFormatter in DateTimeUtils for DATE_FORMAT

2021-11-09 Thread Sergey Nuyanzin
Hi all, In flink-table-common in DateTimeUtils class there are several dateFormat methods. Based on documentation [1] DATE_FORMAT Built-in function it should be compatible with SimpleDateFormat. However in fact it is compatible only for timestamps with non local timezones while for timestamps with

[jira] [Created] (FLINK-24843) DynamicTableFactory.Context.getCatalogTable().getPartitionKeys() should return indexes

2021-11-09 Thread Francesco Guardiani (Jira)
Francesco Guardiani created FLINK-24843: --- Summary: DynamicTableFactory.Context.getCatalogTable().getPartitionKeys() should return indexes Key: FLINK-24843 URL:

[jira] [Created] (FLINK-24842) Make outputs depends on tails for the iteration body

2021-11-09 Thread Yun Gao (Jira)
Yun Gao created FLINK-24842: --- Summary: Make outputs depends on tails for the iteration body Key: FLINK-24842 URL: https://issues.apache.org/jira/browse/FLINK-24842 Project: Flink Issue Type:

[jira] [Created] (FLINK-24841) ChainOrderTest.testMigrationAndRestore fails on AZP

2021-11-09 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-24841: - Summary: ChainOrderTest.testMigrationAndRestore fails on AZP Key: FLINK-24841 URL: https://issues.apache.org/jira/browse/FLINK-24841 Project: Flink Issue

[jira] [Created] (FLINK-24840) JobManagerHAProcessFailureRecoveryITCase crashed on AZP

2021-11-09 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-24840: - Summary: JobManagerHAProcessFailureRecoveryITCase crashed on AZP Key: FLINK-24840 URL: https://issues.apache.org/jira/browse/FLINK-24840 Project: Flink

[jira] [Created] (FLINK-24839) CsvFilesystemStreamSinkITCase.testPart times out on AZP

2021-11-09 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-24839: - Summary: CsvFilesystemStreamSinkITCase.testPart times out on AZP Key: FLINK-24839 URL: https://issues.apache.org/jira/browse/FLINK-24839 Project: Flink

Re: [VOTE] FLIP-187: Adaptive Batch Job Scheduler

2021-11-09 Thread Jingsong Li
+1 (non-binding) This greatly enhances the ease of use of batch jobs. (Parallelism setting is really a challenge) (non-binding: I'm not familiar with runtime, just a general understanding) Best, Jingsong On Tue, Nov 9, 2021 at 3:48 PM David Morávek wrote: > > Thanks for the FLIP, this is