Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-12 Thread weijie guo
Hi Zakelly, Thanks for the proposal! I like this idea and I can see the performance improvements it brings. In the previous reply you mentioned “these APIs are in some newly introduced classes, which are located in a different package name with the original one”. I can see the benefits of this. T

[jira] [Created] (FLINK-34660) AutoRescalingITCase#testCheckpointRescalingInKeyedState AssertionError

2024-03-12 Thread Hangxiang Yu (Jira)
Hangxiang Yu created FLINK-34660: Summary: AutoRescalingITCase#testCheckpointRescalingInKeyedState AssertionError Key: FLINK-34660 URL: https://issues.apache.org/jira/browse/FLINK-34660 Project: Flink

Re: [DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-12 Thread Jark Wu
Hi Minge, Chris, Hao, Thanks for proposing this interesting idea. I think this is a nice step towards the AI world for Apache Flink. I don't know much about AI/ML, so I may have some stupid questions. 1. Could you tell more about why polymorphism table function (PTF) doesn't work and do we have p

Re: Flink's treatment to "hadoop" and "yarn" configuration overrides seems unintuitive

2024-03-12 Thread Venkatakrishnan Sowrirajan
Thanks for your response. Sorry for the late reply, Ferenc. Yes, totally agree with you on deprecating this behavior as part of 1.20. Let me follow it up with a FLIP to deprecate the current behavior and with a proposed solution. We can discuss further in the [DISCUSS] thread of the FLIP. Regards

Re: Question around Flink's AdaptiveBatchScheduler

2024-03-12 Thread Venkatakrishnan Sowrirajan
Thanks for the response Lijie and Junrui. Sorry for the late reply. Few follow up questions. > Source can actually ignore this limit because it has no upstream, but this will lead to semantic inconsistency. Lijie, can you please elaborate on the above comment further? What do you mean when you sa

Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-12 Thread Zakelly Lan
Hi Jing, The deprecation and removal of original APIs is beyond the scope of current FLIP, but I do add/highlight such information under "Compatibility, Deprecation, and Migration Plan" section. Best, Zakelly On Wed, Mar 13, 2024 at 9:18 AM Yunfeng Zhou wrote: > Hi Zakelly, > > Thanks for you

Re: [DISCUSS] FLIP-425: Asynchronous Execution Model

2024-03-12 Thread Yanfei Lei
Hi Jing, Thanks for the reply and follow up. > What is the benefit for users to build a chain of mails instead of just one > mail(it is still async)? Just to make sure we're on the same page, I try to paraphrase your question: A `then()` call will be encapsulated as a callback mail. Your questio

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Qingsheng Ren
+1 (binding) - Verified signature and checksum - Verified no binary in source - Built from source - Tested reading and writing Kafka with SQL client and Kafka connector 3.1.0 - Verified source code tag - Reviewed release note - Reviewed web PR Thanks to all release managers and contributors for

[jira] [Created] (FLINK-34659) How to implement global sort in latest flink datastream API

2024-03-12 Thread Junyao Huang (Jira)
Junyao Huang created FLINK-34659: Summary: How to implement global sort in latest flink datastream API Key: FLINK-34659 URL: https://issues.apache.org/jira/browse/FLINK-34659 Project: Flink

Re: [DISCUSS]FLIP-420: Add API annotations for RocksDB StateBackend user-facing classes

2024-03-12 Thread Jinzhong Li
Hi Jing, Thanks for your feedback! > I am fine with @Internal and once we > use @Internal, we should add it to all classes within a module to keep the > consistency. I think the scope to keep consistency within a module is all interfaces, not all classes. For example, in changelog-statebackend m

Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-12 Thread Yunfeng Zhou
Hi Zakelly, Thanks for your responses. I agree with it that we can keep the design as it is for now and see if others have any better ideas for these questions. Best, Yunfeng On Tue, Mar 12, 2024 at 5:23 PM Zakelly Lan wrote: > > Hi Xuannan, > > Thanks for your comments, I modified the FLIP acc

Re: [VOTE] FLIP-420: Add API annotations for RocksDB StateBackend user-facing classes

2024-03-12 Thread Jing Ge
+1 (binding) Thanks! Best regards, Jing On Sun, Mar 10, 2024 at 5:32 PM Jing Ge wrote: > Hi Jinzhong, > > Thanks for driving this topic and sorry for just joining the discussion > now. I replied in your discussion thread. Would you like to take a look > and let's keep the discussion there? I wi

Re: [DISCUSS]FLIP-420: Add API annotations for RocksDB StateBackend user-facing classes

2024-03-12 Thread Jing Ge
Hi Jinzhong, Thanks for your clarification! > As mentioned in Flink's “API compatibility guarantees”[1], "any API without > such an annotation is considered internal to Flink, with no guarantees > being provided." > I think these interfaces are treated as internal interfaces regardless of > whet

[jira] [Created] (FLINK-34658) Scala API unusable on Flink 1.18.1/Java 17 Docker image

2024-03-12 Thread Matthew Ernst (Jira)
Matthew Ernst created FLINK-34658: - Summary: Scala API unusable on Flink 1.18.1/Java 17 Docker image Key: FLINK-34658 URL: https://issues.apache.org/jira/browse/FLINK-34658 Project: Flink Iss

Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-12 Thread Jing Ge
Hi Zakelly, Thanks for your clarification! I would suggest explicitly adding description(better highlight) into the FLIP that the original State API will be deprecated. My gut feeling is that it is very important for anyone, who will review the new design, to understand the long-term intention. B

Re: [DISCUSS] FLIP-425: Asynchronous Execution Model

2024-03-12 Thread Jing Ge
Hi Yanfei, Thanks for your clarification! Now I got a much clear picture and I am still trying to understand your thoughts for some of those questions: > How many mails are encapsulated depends on how the user writes the > code. The statements in a `then()` will be wrapped into a mail. > StateFu

Additional metadata available for Kafka serdes

2024-03-12 Thread Balint Bene
Hello! Looking to get some guidance for a problem around the Flink formats used for Kafka. Flink currently uses common serdes interfaces across all formats. However, some data formats used in Kafka require headers for serdes. It's the same problem for serialization and deserialization, so I'll ju

[DISCUSS] FLIP-437: Support ML Models in Flink SQL

2024-03-12 Thread Hao Li
Hi, Dev Mingge, Chris and I would like to start a discussion about FLIP-437: Support ML Models in Flink SQL. This FLIP is proposing to support machine learning models in Flink SQL syntax so that users can CRUD models with Flink SQL and use models on Flink to do prediction with Flink data. The FL

Re: [DISCUSS] FLINK-34440 Support Debezium Protobuf Confluent Format

2024-03-12 Thread Kevin Lam
Hi Anupam, Thanks again for your work on contributing this feature back. Sounds good re: the refactoring/re-organizing. Regarding the schema-id, in my opinion this should NOT be a configuration option on the format. We should be able to deterministically map the Flink type to the ProtoSchema and

Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2024-03-12 Thread Gyula Fóra
That would be great Mate! If you could draw up a FLIP for this that would be nice as this is a rather large change that will have a significant impact for existing users. If possible it would be good to provide some backward compatibility / transition period while we preserve the current content o

Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2024-03-12 Thread Mate Czagany
Hi, I really like this idea as well, I think it would be a great improvement compared to how manual savepoints currently work, and suits Kubernetes workflows a lot better. If there are no objections, I can investigate it during the next few weeks and see how this could be implemented in the curre

[jira] [Created] (FLINK-34657) Implement Lineage Graph for streaming API use cases

2024-03-12 Thread Zhenqiu Huang (Jira)
Zhenqiu Huang created FLINK-34657: - Summary: Implement Lineage Graph for streaming API use cases Key: FLINK-34657 URL: https://issues.apache.org/jira/browse/FLINK-34657 Project: Flink Issue T

Re: [DISCUSS] FLIP-435: Introduce a New Dynamic Table for Simplifying Data Pipelines

2024-03-12 Thread Timo Walther
Hi Lincoln & Ron, thanks for proposing this FLIP. I think a design similar to what you propose has been in the heads of many people, however, I'm wondering how this will fit into the bigger picture. I haven't deeply reviewed the FLIP yet, but would like to ask some initial questions: Flink

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Matthias Pohl
I want to share an update on FLINK-34227 [1]: It's still not clear what's causing the test instability. So far, we agreed in today's release sync [2] that it's not considered a blocker because it is observed in 1.18 nightly builds and it only appears in the GitHub Actions workflow. But I still have

Re: Default scale and precision SQL data types

2024-03-12 Thread Timo Walther
Hi Sergei, please check with the SQL standard before. Most of these values have been derived from the standard. I don't like the default of TIMESTAMP(6) for timestamps but this is what the standard dictates. Same for not allowing VARCHAR(0) or VARCHAR defaulting to VARCHAR(1). Changes to the

Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2024-03-12 Thread Gyula Fóra
That's definitely a good improvement Robert and we should add it at some point. At the point in time when this was implemented we went with the current simpler / more lightweight approach. However if anyone is interested in working on this / contributing this improvement I would personally support

Re: [DISCUSS] Manual savepoint triggering in flink-kubernetes-operator

2024-03-12 Thread Robert Metzger
Have you guys considered making savepoints a first class citizen in the Kubernetes operator? E.g. to trigger a savepoint, you create a "FlinkSavepoint" CR, the K8s operator picks up that resource and tries to create a savepoint indefinitely until the savepoint has been successfully created. We repo

[jira] [Created] (FLINK-34656) Generated code for `ITEM` operator should return null when getting element of a null map/array/row

2024-03-12 Thread yisha zhou (Jira)
yisha zhou created FLINK-34656: -- Summary: Generated code for `ITEM` operator should return null when getting element of a null map/array/row Key: FLINK-34656 URL: https://issues.apache.org/jira/browse/FLINK-34656

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Benchao Li
+1 (non-binding) - checked signature and checksum: OK - checkout copyright year in notice file: OK - diffed source distribution with tag, make sure there is no unexpected files: OK - build from source : OK - start a local cluster, played with jdbc connector: OK weijie guo 于2024年3月12日周二 16:55写道:

[jira] [Created] (FLINK-34655) Autoscaler doesn't work for flink 1.15

2024-03-12 Thread Rui Fan (Jira)
Rui Fan created FLINK-34655: --- Summary: Autoscaler doesn't work for flink 1.15 Key: FLINK-34655 URL: https://issues.apache.org/jira/browse/FLINK-34655 Project: Flink Issue Type: Bug Compon

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu
I have created a JIRA issue and opened a pull request for this: https://github.com/apache/flink-web/pull/725. Best, Jark On Tue, 12 Mar 2024 at 16:56, Jark Wu wrote: > Thank you all for your feedback. If there are no other concerns or > objections, > I'm going to create a pull request to add th

Re: [DISCUSS] FLIP-433: State Access on DataStream API V2

2024-03-12 Thread Zakelly Lan
Hi Weijie, Thanks for your reply! Overall I'd be fine with the builder pattern, but it is a little bit long when carrying explicit 'build()' and declaring the builder. Keeping the StateDeclaration immutable is OK, but it is a little bit inconvenient for overriding the undefined options by job con

Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-12 Thread Zakelly Lan
Hi Xuannan, Thanks for your comments, I modified the FLIP accordingly. Hi Yunfeng, Thanks for sharing your opinions! Could you provide some hint on use cases where users need to mix sync > and async state operations in spite of the performance regression? > This information might help address o

[jira] [Created] (FLINK-34654) Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu (Jira)
Jark Wu created FLINK-34654: --- Summary: Add "Special Thanks" Page on the Flink Website Key: FLINK-34654 URL: https://issues.apache.org/jira/browse/FLINK-34654 Project: Flink Issue Type: New Feature

Re: [DISCUSS] Add "Special Thanks" Page on the Flink Website

2024-03-12 Thread Jark Wu
Thank you all for your feedback. If there are no other concerns or objections, I'm going to create a pull request to add the Special Thanks page. Further feedback and sponsors to be added are still welcome! Best, Jark On Mon, 11 Mar 2024 at 23:09, Maximilian Michels wrote: > Hi Jark, > > Thank

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread weijie guo
+1 (non-binding) - Verified signature and checksum - Verified source distribution does not contains binaries - Build from source code and submit a word-count job successfully Best regards, Weijie Jane Chan 于2024年3月12日周二 16:38写道: > +1 (non-binding) > > - Verify that the source distributions

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Rui Fan
+1 (non-binding) - Verified signature - Verified checksum - Built source code successfully - Reviewed the release PR and some comments are updated Best, Rui On Tue, Mar 12, 2024 at 4:08 PM Xuannan Su wrote: > +1 (non-binding) > > - Verified signature and checksum > - Verified that source distr

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Jane Chan
+1 (non-binding) - Verify that the source distributions do not contain any binaries; - Build the source distribution to ensure all source files have Apache headers; - Verify checksum and GPG signatures; Best, Jane On Tue, Mar 12, 2024 at 4:08 PM Xuannan Su wrote: > +1 (non-binding) > > - Verif

Re: [VOTE] Release 1.19.0, release candidate #2

2024-03-12 Thread Xuannan Su
+1 (non-binding) - Verified signature and checksum - Verified that source distribution does not contain binaries - Built from source code successfully - Reviewed the release announcement PR Best regards, Xuannan On Tue, Mar 12, 2024 at 2:18 PM Hang Ruan wrote: > > +1 (non-binding) > > - Verifie

Re: [DISCUSS] FLIP-434: Support optimizations for pre-partitioned data sources

2024-03-12 Thread Jane Chan
Hi Jeyhun, Thank you for leading the discussion. I'm generally +1 with this proposal, along with some questions. Please see my comments below. 1. Concerning the `sourcePartitions()` method, the partition information returned during the optimization phase may not be the same as the partition infor

[jira] [Created] (FLINK-34653) Support table merging with route in Flink CDC

2024-03-12 Thread Qingsheng Ren (Jira)
Qingsheng Ren created FLINK-34653: - Summary: Support table merging with route in Flink CDC Key: FLINK-34653 URL: https://issues.apache.org/jira/browse/FLINK-34653 Project: Flink Issue Type: N