[DISCUSS] Improvements to SQL State Compatibility by Increasing Schema Evolution Capabilities

2025-02-03 Thread Eric Xiao
Hi community, When customers are iterating on the Flink SQL pipelines, we have discovered a couple of state incompatible changes that we believe could be state compatible and would allow working with stateful Flink SQL pipeline easier. Today some of these incompatibility issues can be addressed b

[jira] [Created] (FLINK-37240) Changing column nullability throws StateMigrationException in a SQL pipeline with a JOIN

2025-01-30 Thread Eric Xiao (Jira)
Eric Xiao created FLINK-37240: - Summary: Changing column nullability throws StateMigrationException in a SQL pipeline with a JOIN Key: FLINK-37240 URL: https://issues.apache.org/jira/browse/FLINK-37240

Re: [DISCUSS] FLIP-XXX: Blue/Green Deployments for Flink on Kubernetes: Phase 1 (basic)

2025-01-24 Thread Eric Xiao
Hi Sergio, Can you update the Phase 1 Google Doc's sharing permissions? I also believe the link in the FLIP leads to an internal Apple tool: https://quip-apple.com/account/login?next=https%3A%2F%2Fquip-apple.com%2F7BpiAdeZ7Ow3 On Tue, Jan 14, 2025 at 12:15 PM Sergio Chong Loo wrote: > FLIP-503:

Re: [DISCUSS] FLIP-487: Show history of rescales in Web UI for AdaptiveScheduler

2024-12-06 Thread Eric Xiao
Hi Yuepeng Pan, This would be a very useful feature to have in OSS. One question I have that wasn't outlined in the FLIP or I may have missed it was - How/Where would we store the history of these scaling events? Is there an existing pattern for storing such information in other parts of Flink th

Re: Stateful AsyncIO operator

2024-08-09 Thread Eric Xiao
Hey Taher, To answer question 2 and 3 - per the flink documentation[1] in-flight requests are stored in checkpoints and only the unsent records will be sent when recovering from a failure. https://nightlies.apache.org/flink/flink-docs-master/docs/dev/datastream/operators/asyncio/#fault-tolerance-

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

2023-04-25 Thread eric xiao
hink 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-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

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

2023-04-21 Thread eric xiao
Best, 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/datas

[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

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

[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

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

2022-10-28 Thread eric xiao
e 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/t

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

2022-10-28 Thread eric xiao
1), None)] > > +- OneRowRelation > > ` > > > > > > 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. Bu

[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

[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