[DISCUSS] FLINK-20578 Cannot create empty array using ARRAY[]

2022-10-27 Thread eric xiao
Hi, I would like to propose a solution to this JIRA issue. I looked at the comments and there was some guidance around where in the code we should update to allow for this behaviour. But I believe there are still two questions that remain open: 1. Is this expected behaviour (i.e. users should

Re: [DISCUSS] FLINK-20578 Cannot create empty array using ARRAY[]

2022-10-28 Thread eric xiao
r the keyword 'INT' at line 1, column 8. Was expecting one of: "ABS" ... "ALL" ... "ARRAY" ... "AVG" ... "CARDINALITY" ... Am I missing something? [1] https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/types/#a

Re: [DISCUSS] FLINK-20578 Cannot create empty array using ARRAY[]

2022-10-28 Thread eric xiao
; > > Seems it's hard to decide which data type Flink should use. I'm > insterested in the reason why you would like to use Integer type. > > I haven't cheked whether the sql stardard specifies it. But from my > side, I prefer to follow Hive/Spark. > > > > BTW: the quer

Re: [DISCUSS] FLINK-20578 Cannot create empty array using ARRAY[]

2022-11-03 Thread eric xiao
-L114 On Fri, Oct 28, 2022 at 10:17 PM yuxia wrote: > `ARRAY`/ `INT ARRAY` is for declaring data type in DDL like `create > table t1(a INT ARRAY)`. > > Best regards, > Yuxia > > - 原始邮件 ----- > 发件人: "eric xiao" > 收件人: "dev" > 发送时间: 星期六, 2

[DISCUSS] FLINK-31873: Add setMaxParallelism to the DataStreamSink Class

2023-04-21 Thread eric xiao
, Eric Xiao [1] https://issues.apache.org/jira/browse/FLINK-31873 [2] https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/deployment/elastic_scaling/#configuration [3] https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream

Re: [DISCUSS] FLINK-31873: Add setMaxParallelism to the DataStreamSink Class

2023-04-25 Thread eric xiao
n. > > > I think it's reasonable to add ''setMaxParallelism" for DataStreamSink. > > > > > > +1 > > > > > > Best, > > > Weihua > > > > > > > > > On Sat, Apr 22, 2023 at 3:20 AM eric xiao > wrote: > > > > > > >

[jira] [Created] (FLINK-29498) Flink Async I/O Retry Strategies Do Not Work for Scala AsyncDataStream API

2022-10-03 Thread Eric Xiao (Jira)
Eric Xiao created FLINK-29498: - Summary: Flink Async I/O Retry Strategies Do Not Work for Scala AsyncDataStream API Key: FLINK-29498 URL: https://issues.apache.org/jira/browse/FLINK-29498 Project: Flink

[jira] [Created] (FLINK-29837) SQL API does not expose the RowKind of the Row for processing Changelogs

2022-11-01 Thread Eric Xiao (Jira)
Eric Xiao created FLINK-29837: - Summary: SQL API does not expose the RowKind of the Row for processing Changelogs Key: FLINK-29837 URL: https://issues.apache.org/jira/browse/FLINK-29837 Project: Flink

[jira] [Created] (FLINK-31873) Add setMaxParallelism to the DataStreamSink Class

2023-04-20 Thread Eric Xiao (Jira)
Eric Xiao created FLINK-31873: - Summary: Add setMaxParallelism to the DataStreamSink Class Key: FLINK-31873 URL: https://issues.apache.org/jira/browse/FLINK-31873 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-31941) Not backwards compatible naming for some kubernetes resources

2023-04-25 Thread Eric Xiao (Jira)
Eric Xiao created FLINK-31941: - Summary: Not backwards compatible naming for some kubernetes resources Key: FLINK-31941 URL: https://issues.apache.org/jira/browse/FLINK-31941 Project: Flink