Re: [ANNOUNCE] New Apache Flink PMC Member - Fan Rui

2024-06-10 Thread Jeyhun Karimov
Congratulations Rui, well deserved! Regards, Jeyhun On Mon, Jun 10, 2024, 10:21 Ahmed Hamdy wrote: > Congratulations Rui! > Best Regards > Ahmed Hamdy > > > On Mon, 10 Jun 2024 at 09:10, David Radley > wrote: > > > Congratulations, Rui! > > > > From: Sergey Nuyanzin > > Date: Sunday, 9 June

Re: [DISCUSS] FLIP-463: Schema Definition in CREATE TABLE AS Statement

2024-06-10 Thread Jeyhun Karimov
Hi Sergio, Thanks for driving this FLIP. +1 for it. I have a few questions: - How do you handle CTAS statements with SELECT clauses that have (implicit or explicit) NULLABLE or NOT NULLABLE columns? Could you also mention the casting rules in the FLIP for this case? - In some cases, redefining

Re: [VOTE] FLIP-459: Support Flink hybrid shuffle integration with Apache Celeborn

2024-06-08 Thread Jeyhun Karimov
Hi Yuxin, Thanks for driving this. +1 (non-binding) Regards, Jeyhun On Fri, Jun 7, 2024 at 6:05 PM Jim Hughes wrote: > HI all, > > +1 (non-binding) > > Cheers, > > Jim > > On Fri, Jun 7, 2024 at 4:03 AM Yuxin Tan wrote: > > > Hi everyone, > > > > Thanks for all the feedback about the

Re: [VOTE] Release 1.19.1, release candidate #1

2024-06-08 Thread Jeyhun Karimov
Hi Hong, Thanks for driving the release. +1 (non-binding) - Verified gpg signature - Reviewed the PR - Verified sha512 - Checked github release tag - Checked that the source code does not contain binaries Regards, Jeyhun On Sat, Jun 8, 2024 at 1:52 PM weijie guo wrote: > Thanks Hong! > >

Re: [DISCUSS] FLIP-459: Support Flink hybrid shuffle integration with Apache Celeborn

2024-06-06 Thread Jeyhun Karimov
Hi Yuxin, +1 for this proposal. This change will greatly alleviate the pressure on local storage resources (especially when there is limited local storage) particularly in the context of cloud-native environments. Regards, Jeyhun On Thu, Jun 6, 2024 at 1:20 PM Yuxin Tan wrote: > Hi all, > >

Re: [DISCUSS] Add Flink CDC Channel to Apache Flink Slack Workspace

2024-06-04 Thread Jeyhun Karimov
Thanks a lot Zhongqiang for the proposal (+1) and Leonard for creating the channel! Regards, Jeyhun On Tue, Jun 4, 2024 at 6:12 AM Leonard Xu wrote: > I’ve created flink-cdc channel in Apache Flink Slack Workspace via > https://issues.apache.org/jira/browse/FLINK-35514 < >

Re: Re: [ANNOUNCE] New Apache Flink PMC Member - Weijie Guo

2024-06-04 Thread Jeyhun Karimov
Congratulations Weijie! Well deserved. Regards, Jeyhun On Tue, Jun 4, 2024 at 3:50 PM Feng Jin wrote: > Congratulations, Weijie! > > > Best, > Feng Jin > > On Tue, Jun 4, 2024 at 9:03 PM Wencong Liu wrote: > > > Congratulations, Weijie! > > > > > > > > > > Best, > > > > Wencong > > > > > > >

Re: [DISCUSS] Merge "flink run" and "flink run-application" in Flink 2.0

2024-06-03 Thread Jeyhun Karimov
Hi Ferenc, Thanks for the proposal. +1 for it! This FLIP will improve the user experience. Regards, Jeyhun On Mon, Jun 3, 2024 at 1:50 PM Ferenc Csaky wrote: > Hi Hang, > > Thank you for your inputs, both points make sense, updated the > FLIP according to them. > > Best, > Ferenc > > > >

Re: [DISCUSS] FLIP-XXX: Improve JDBC connector extensibility for Table API

2024-05-14 Thread Jeyhun Karimov
Hi Lorenzo, Thanks for driving this FLIP. +1 for it. Could you please elaborate more on how the new approach will be backwards compatible? Regards, Jeyhun On Tue, May 14, 2024 at 10:00 AM Muhammet Orazov wrote: > Hey Lorenzo, > > Thanks for driving this FLIP! +1 > > It will improve the user

Re: [VOTE] FLIP-451: Introduce timeout configuration to AsyncSink

2024-05-13 Thread Jeyhun Karimov
Thanks for driving this Ahmed. +1 (non-binding) Regards, Jeyhun On Mon, May 13, 2024 at 12:37 PM Muhammet Orazov wrote: > Thanks Ahmed, +1 (non-binding) > > Best, > Muhammet > > On 2024-05-13 09:50, Ahmed Hamdy wrote: > > Hi all, > > > > Thanks for the feedback on the discussion thread[1], I

Re: Access to Wiki for raising a Flip request

2024-05-13 Thread Jeyhun Karimov
Hi Vikas, Thanks for your request. According to the new updates from ASF infra, new users can't signup and access Confluence to create FLIPs. As initiated by Martijn and discussed by the community [1], the current approach for the new users is to create a Google Doc and make that view-only, and

Re: [DISCUSSION] FLIP-450: Improve Runtime Configuration for Flink 2.0

2024-05-13 Thread Jeyhun Karimov
he FLIP to include the information. > > Best regards, > Xuannan > > On Sun, May 12, 2024 at 9:16 PM Jeyhun Karimov > wrote: > > > > Hi Xuannan, > > > > Thanks for driving this FLIP! > > I have a minor comment. Do we plan to remove StreamPipelineOptions in

Re: [DISCUSSION] FLIP-450: Improve Runtime Configuration for Flink 2.0

2024-05-12 Thread Jeyhun Karimov
Hi Xuannan, Thanks for driving this FLIP! I have a minor comment. Do we plan to remove StreamPipelineOptions in 2.0, as it only contains deprecated options? Regards, Jeyhun On Sat, May 11, 2024 at 4:40 AM Rui Fan <1996fan...@gmail.com> wrote: > Thanks Xuannan for the update! > > LGTM, +1 for

Re: Alignment with FlinkSQL Standards | Feedback requested

2024-05-10 Thread Jeyhun Karimov
Dear Kanchi, Thanks for your proposal. I think a similar feature is available in Flink Pulsar Connector. So, from the sql semantics and philosophy there should not be any issues IMHO. Could you please elaborate more on the motivation (e.g., user requests, enabling new use-cases for Kafka, etc)

Re: [VOTE] FLIP-449: Reorganization of flink-connector-jdbc

2024-05-10 Thread Jeyhun Karimov
Thanks for driving this! +1 (non-binding) Regards, Jeyhun On Fri, May 10, 2024 at 12:50 PM Muhammet Orazov wrote: > Thanks João for your efforts and driving this! > > +1 (non-binding) > > Best, > Muhammet > > On 2024-05-09 12:01, Joao Boto wrote: > > Hi everyone, > > > > Thanks for all the

Re: [VOTE] Apache Flink CDC Release 3.1.0, release candidate #1

2024-05-03 Thread Jeyhun Karimov
Hi Qinsheng, Thanks for driving the release. +1 (non-binding) - No binaries in source - Verified Signatures - Github tag exists - Build source Regards, Jeyhun On Thu, May 2, 2024 at 10:52 PM Muhammet Orazov wrote: > Hey Qingsheng, > > Thanks a lot! +1 (non-binding) > > - Checked sha512sum

Re: Discussion: Condition field in the CR status

2024-05-03 Thread Jeyhun Karimov
Hi Lajith, Thanks a lot for driving this FLIP. Please find my comments below: - I echo Gyula that including some examples and further explanations might ease reader's work. With the current version, the FLIP is a bit hard to follow. - Will the usage of Conditions be enabled by default? Or will

Re: [VOTE] FLIP-454: New Apicurio Avro format

2024-05-03 Thread Jeyhun Karimov
+1 (non binding) Thanks for driving this FLIP David. Regards, Jeyhun On Fri, May 3, 2024 at 2:21 PM Mark Nuttall wrote: > +1, I would also like to see first class support for Avro and Apicurio > > -- Mark Nuttall, mnutt...@apache.org > Senior Software Engineer, IBM Event Automation > > On

Re: [DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc

2024-05-03 Thread Jeyhun Karimov
Hi Boto, Thanks for driving this FLIP. +1 for it. I would also ask to include a sample usage and changes for end-users in the FLIP. flink-connector-jdbc: The current module, which will be transformed to > shade all other modules and maintain backward compatibility. Also, in order to ensure

Re: [DISCUSS] FLIP-451: Refactor Async sink API

2024-04-29 Thread Jeyhun Karimov
Hi Ahmed, Thanks a lot for the FLIP. +1 for it. My main concern is that the boundary/scope of the two FLIPs (451 and 284) and their differentiation/overlap is unclear for me from the FLIP document. Could you please elaborate more on this? Regards, Jeyhun On Mon, Apr 29, 2024 at 4:13 PM Ahmed

Re: [VOTE] Release flink-connector-mongodb v1.2.0, release candidate #1

2024-04-17 Thread Jeyhun Karimov
Thanks for driving this. +1 (non-binding) - Validated checksum hash - Verified signature - Tag is present - Reviewed web PR Regards, Jeyhun On Wed, Apr 17, 2024 at 3:26 PM gongzhongqiang wrote: > +1 (non-binding) > > - Flink website pr reviewed > - Check source code without binary files > -

Re: [VOTE] FLIP-442: General Improvement to Configuration for Flink 2.0

2024-04-17 Thread Jeyhun Karimov
+1 (non binding) Regards, Jeyhun On Wed, Apr 17, 2024 at 2:22 PM Zhu Zhu wrote: > +1 (binding) > > Thanks, > Zhu > > Yuxin Tan 于2024年4月17日周三 18:36写道: > > > +1 (non-binding) > > > > Best, > > Yuxin > > > > > > Zakelly Lan 于2024年4月17日周三 16:51写道: > > > > > +1 binding > > > > > > > > > Best, > >

Re: [ANNOUNCE] New Apache Flink Committer - Zakelly Lan

2024-04-16 Thread Jeyhun Karimov
Congratulations Zakelly! Regards, Jeyhun On Tue, Apr 16, 2024 at 6:35 AM Feifan Wang wrote: > Congratulations, Zakelly!—— > > Best regards, > > Feifan Wang > > > > > At 2024-04-15 10:50:06, "Yuan Mei" wrote: > >Hi everyone, > > > >On behalf of the PMC, I'm happy to let you know

Re: [DISCUSS] FLIP-445: Support dynamic parallelism inference for HiveSource

2024-04-16 Thread Jeyhun Karimov
Hi Xia, Thanks for driving this FLIP. +1 from my side. I have one comment. Could you please mention the default values for the new configurations (e.g., table.exec.hive.infer-source-parallelism.mode, table.exec.hive.infer-source-parallelism.enabled, etc) ? Regards, Jeyhun On Tue, Apr 16, 2024

Re: [Vote] FLIP-438: Amazon SQS Sink Connector

2024-04-16 Thread Jeyhun Karimov
Thanks Priya for driving the FLIP. +1 (non-binding) Regards, Jeyhun On Tue, Apr 16, 2024 at 12:37 PM Hong Liang wrote: > +1 (binding) > > Thanks Priya for driving this! This has been a requested feature for a > while now, and will benefit the community :) > > Hong > > On Tue, Apr 16, 2024 at

Re: [VOTE] Release flink-connector-aws v4.3.0, release candidate #1

2024-04-16 Thread Jeyhun Karimov
+1 (non-binding) - Verified tags - Verified Lisence - Reviewed web pr Regards, Jeyhun On Tue, Apr 16, 2024 at 12:59 PM Danny Cranmer wrote: > Hi everyone, > > Please review and vote on the release candidate #1 for flink-connector-aws > v4.3.0, as follows: > [ ] +1, Approve the release > [ ]

Re: [ANNOUNCE] New Apache Flink PMC Member - Lincoln Lee

2024-04-12 Thread Jeyhun Karimov
Congratulations, Lincoln! Regards, Jeyhun On Fri, Apr 12, 2024 at 12:31 PM Aleksandr Pilipenko wrote: > Congratulations, Lincoln! > > Best Regards > Aleksandr >

Re: [ANNOUNCE] New Apache Flink PMC Member - Jing Ge

2024-04-12 Thread Jeyhun Karimov
Congratulations, Jing! Regards, Jeyhun On Fri, Apr 12, 2024 at 12:28 PM Aleksandr Pilipenko wrote: > Congratulations, Jing! > > Best Regards, > Aleksandr >

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

2024-04-09 Thread Jeyhun Karimov
ator) still relies on the KeyedStream, meaning > that consistency of the partitioner and state key selector is still > required. > > Best, > Lincoln Lee > > > Jeyhun Karimov 于2024年4月6日周六 05:11写道: > > > Hi Lincoln, > > > > I did a bit of analysis on sma

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

2024-04-05 Thread Jeyhun Karimov
is a good > > candidates for streaming source example, file source is a good one for > > batch source and it really helped me to follow-up the FLIP. > > > > Best, > > Leonard > > [1] > > > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache

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

2024-04-05 Thread Jeyhun Karimov
e instances) is always deterministic under the same Kafka configuration. Hope that answers your question. Regards, Jeyhun On Wed, Apr 3, 2024 at 6:53 PM Jeyhun Karimov wrote: > Hi Leonard, > > Thanks a lot for your comments. Please find my answers below: > > (1)The FLIP motivation

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

2024-04-03 Thread Jeyhun Karimov
me to follow-up the FLIP. > > > > Best, > > Leonard > > [1] > > > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java#L31 > > > > > > > > > 2024年4月3日 上午5:53,Jeyh

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

2024-04-03 Thread Jeyhun Karimov
Source is a good > candidates for streaming source example, file source is a good one for > batch source and it really helped me to follow-up the FLIP. > > Best, > Leonard > [1] > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/produce

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

2024-04-02 Thread Jeyhun Karimov
doing this is to > directly reuse the > current shuffle remove optimization (as `FlinkExpandConversionRule` > implemented), > and according to the data distribution characteristics provided by the > source, the planner > may choose a physical operator with a cheaper costs (for exa

Re: [DISCUSS] FLIP-XXX: Introduce Flink SQL variables

2024-04-01 Thread Jeyhun Karimov
Hi Ferenc, Thanks for the proposal. Sounds like a good idea! I have a few questions on that: - What is its impact on query optimization because resolving variables at the parsing stage might affect query optimization. - What is the scope of variables? I mean when and how they override each

Re: Re: [ANNOUNCE] Apache Paimon is graduated to Top Level Project

2024-04-01 Thread Jeyhun Karimov
Congratulations! Regards, Jeyhun On Mon, Apr 1, 2024 at 7:43 AM Guowei Ma wrote: > Congratulations! > Best, > Guowei > > > On Mon, Apr 1, 2024 at 11:15 AM Feng Jin wrote: > > > Congratulations! > > > > Best, > > Feng Jin > > > > On Mon, Apr 1, 2024 at 10:51 AM weijie guo > > wrote: > > > >>

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

2024-04-01 Thread Jeyhun Karimov
the >> code that partitions exist. >> >> Another thing is that you show Hive-like partitioning in your FS >> structure, do you think it makes sense to add a note about auto-discovery >> of partitions? >> >> In other terms, it looks a bit counterintuitive that t

Re: [VOTE] FLIP-428: Fault Tolerance/Rescale Integration for Disaggregated State

2024-03-31 Thread Jeyhun Karimov
+1 (non-binding) Regards, Jeyhun On Wed, Mar 27, 2024 at 12:31 PM Jinzhong Li wrote: > Hi devs, > > > I'd like to start a vote on the FLIP-428: Fault Tolerance/Rescale > Integration for Disaggregated State [1]. The discussion thread is here [2]. > > > The vote will be open for at least 72

[jira] [Created] (FLINK-34924) Support partition pushdown for join queries

2024-03-23 Thread Jeyhun Karimov (Jira)
Jeyhun Karimov created FLINK-34924: -- Summary: Support partition pushdown for join queries Key: FLINK-34924 URL: https://issues.apache.org/jira/browse/FLINK-34924 Project: Flink Issue Type

Re: [DISCUSS] FLIP-XXX Apicurio-avro format

2024-03-22 Thread Jeyhun Karimov
a.java#L109 > Then this should work. > > Does this address your question? > Kind regards, David. > > > From: Jeyhun Karimov > Date: Thursday, 21 March 2024 at 21:06 > To: dev@flink.apache.org > Subject: [EXTERNAL] Re: [DISCUSS] FLIP-XXX Apicurio-avro format > Hi David,

Re: [DISCUSS] FLIP-428: Fault Tolerance/Rescale Integration for Disaggregated State

2024-03-21 Thread Jeyhun Karimov
Hi Jinzhong, Thanks for the FLIP. +1 for it. I have a few questions: - Why dont we consider an option where checkpoint directory just contains metadata. So, we do not need to copy the data all the time from working directory to the checkpointing directory. Basically, when checkpointing, 1) we

Re: [DISCUSS] FLIP-XXX Apicurio-avro format

2024-03-21 Thread Jeyhun Karimov
Hi David, Thanks for the FLIP. +1 for it. I have a minor comment. Can you please elaborate more on mechanisms in place to ensure data consistency and integrity, particularly in the event of schema conflicts? Since each message includes a schema ID for inbound and outbound messages, can you

Re: [ANNOUNCE] Donation Flink CDC into Apache Flink has Completed

2024-03-21 Thread Jeyhun Karimov
Great news! Congratulations! Regards, Jeyhun On Thu, Mar 21, 2024 at 2:00 PM Yuxin Tan wrote: > Congratulations! Thanks for the efforts. > > > Best, > Yuxin > > > Samrat Deb 于2024年3月21日周四 20:28写道: > > > Congratulations ! > > > > Bests > > Samrat > > > > On Thu, 21 Mar 2024 at 5:52 PM, Ahmed

Re: [VOTE] FLIP-439: Externalize Kudu Connector from Bahir

2024-03-21 Thread Jeyhun Karimov
+1 (non-binding) Regards, Jeyhun On Thu, Mar 21, 2024 at 2:04 PM Márton Balassi wrote: > +1(binding) > > On Thu, Mar 21, 2024 at 1:24 PM Leonard Xu wrote: > > > +1(binding) > > > > Best, > > Leonard > > > > > 2024年3月21日 下午5:21,Martijn Visser 写道: > > > > > > +1 (binding) > > > > > > On Thu,

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

2024-03-21 Thread Jeyhun Karimov
exist statically (and they can > change at runtime), while the source itself knows the data provider and can > directly implement a method `discoverPartitions`. Then Flink would take > care of invoking that method when needed. > On Mar 15, 2024 at 22:09 +0100, Jeyhun Karimov , > wrote: >

Re: [ANNOUNCE] Apache Flink 1.19.0 released

2024-03-18 Thread Jeyhun Karimov
Congrats! Thanks to release managers and everyone involved. Regards, Jeyhun On Mon, Mar 18, 2024 at 9:25 AM Lincoln Lee wrote: > The Apache Flink community is very happy to announce the release of Apache > Flink 1.19.0, which is the fisrt release for the Apache Flink 1.19 series. > > Apache

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

2024-03-15 Thread Jeyhun Karimov
t; > Another thing is I'm curious about what the physical plan looks like. Is > > there any specific info that will be added to the table source (like > > filter/project pushdown)? It would be great if you could attach an > example > > to the FLIP. > > > &g

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

2024-03-14 Thread Jeyhun Karimov
ould attach an example > to the FLIP. > > Bests, > Jane > > On Wed, Mar 13, 2024 at 9:11 PM Jeyhun Karimov > wrote: > > > Hi Jane, > > > > Thanks for your comments. > > > > > > 1. Concerning the `sourcePartitions()` method, the partition

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

2024-03-14 Thread Jeyhun Karimov
contains > the same partition keys as the source` be changed as `Check if upstream_any > is pre-partitioned data source AND contains the same partition keys as > downstream_any` ? > > Best, > Hang > > Jeyhun Karimov 于2024年3月13日周三 21:11写道: > > > Hi Jane, > >

Re: [DISCUSS] FLIP-436: Introduce "SHOW CREATE CATALOG" Syntax

2024-03-13 Thread Jeyhun Karimov
Hi Yubin, Thanks for the proposal. +1 for it. I have one comment: I would like to see the SQL syntax for the proposed statement. Showing the SQL parser implementation in the FLIP for the SQL syntax might be a bit confusing. Also, the formal definition is missing for this SQL clause. Maybe

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

2024-03-13 Thread Jeyhun Karimov
or Flink, with FLIP-434, the proposed planner rule > > could remove the shuffle. > > > > Definitely a fun idea; I look forward to hearing more! > > > > Cheers, > > > > Jim > > > > > > 1. > > > > > https://cwiki.apache.or

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

2024-03-13 Thread Jeyhun Karimov
could remove the shuffle. > > Definitely a fun idea; I look forward to hearing more! > > Cheers, > > Jim > > > 1. > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-376%3A+Add+DISTRIBUTED+BY+clause > 2. > > https://docs.ksqldb.io/en/latest/develop

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

2024-03-11 Thread Jeyhun Karimov
+1 (non binding) - verified that source distribution does not contain binaries - verified signatures and checksums - built source code successfully Regards, Jeyhun On Mon, Mar 11, 2024 at 3:08 PM Samrat Deb wrote: > +1 (non binding) > > - verified signatures and checksums > - ASF headers are

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

2024-03-10 Thread Jeyhun Karimov
Hi devs, I’d like to start a discussion on FLIP-434: Support optimizations for pre-partitioned data sources [1]. The FLIP introduces taking advantage of pre-partitioned data sources for SQL/Table API (it is already supported as experimental feature in DataStream API [2]). Please find more

Re: [DISCUSS] FLIP-419: Optimize multi-sink query plan generation

2024-03-10 Thread Jeyhun Karimov
reported by mvn test (and not the runtime of the whole mvn test command). I also included the above information in the FLIP. Please let me know if you have further questions. Regards, Jeyhun On Fri, Mar 8, 2024 at 12:01 PM Martijn Visser wrote: > Hi Jeyhun Karimov, > > I see th

Re: [DISCUSS] FLIP-427: Disaggregated State Store

2024-03-10 Thread Jeyhun Karimov
Hi Hangxiang, Thanks for the proposal. +1 for it. I have a few comments. Proposal 2 has additional JNI overhead, but the overhead is relatively > negligible when weighed against the latency of remote I/O. - Is this argument true for all workloads? Or does this argument also hold for workloads

Re: [DISCUSS] FLIP-426: Grouping Remote State Access

2024-03-09 Thread Jeyhun Karimov
Hi Jinzhong, +1 for the FLIP. I have the following comments: - Do we have a fallback mechanism for filesystems that do not support multiget? - Also, in the case of multiget what is the granularity of error handling or retry semantics (e.g., one subrequest fails in multiget). Do we fully rely

Re: [DISCUSS] Support the Ozone Filesystem

2024-03-09 Thread Jeyhun Karimov
Hi Ferenc, Thanks a lot for the proposal. I agree with Martijn. Maybe you can search publicly available sources (e.g., MLs of different projects, stackoverflow, etc), to make sure that there is an organic need for this FS support. That would also help the Flink community to evaluate the proposal

Re: [DISCUSS] FLIP-399: Flink Connector Doris

2024-03-07 Thread Jeyhun Karimov
java#L55 > > > [2] > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-372%3A+Enhance+and+synchronize+Sink+API+to+match+the+Source+API > > > [3] > > > > > > https://github.com/apache/doris-flink-connector/blob/43e0e5cf9b832854ea228

Re: [DISCUSS] FLIP Suggestion: Externalize Kudu Connector from Bahir

2024-03-06 Thread Jeyhun Karimov
Hi Ferenc, +1 for the FLIP. One minor comment. As part of the externalization process, the DataStream connector > implementation can be updated to the new Source/Sink APIs - How do you plan the review process in this case (e.g. incremental over existing codebase or cumulative all at once) ?

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

2024-03-06 Thread Jeyhun Karimov
Hi Jinzhong, Thanks for the FLIP. +1 (non-binding) Regards, Jeyhun On Wed, Mar 6, 2024 at 5:09 PM Yun Tang wrote: > +1 for this FLIP. > > > > Best > Yun Tang > > From: Jinzhong Li > Sent: Wednesday, March 6, 2024 20:29 > To: dev@flink.apache.org > Subject:

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-06 Thread Jeyhun Karimov
amount. > > > >> > Also control the max StateRequests that can be executed in > parallel > > for each StreamRecord, and if it exceeds, put the new StateRequest in the > > blocking buffer waiting for execution (instead of invoking yield()). > > > >&g

Re: [DISCUSS] FLIP-399: Flink Connector Doris

2024-03-02 Thread Jeyhun Karimov
Hi, Thanks for the proposal. +1 for the FLIP. I have a few questions: - How exactly the two (Stream Load's two-phase commit and Flink's two-phase commit) combination will ensure the e2e exactly-once semantics? - The FLIP proposes to combine Doris's batch writing with the primary key table to

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-01 Thread Jeyhun Karimov
Hi, Thanks for the great proposals. I have a few comments comments: - Backpressure Handling. Flink's original backpressure handling is quite robust and the semantics is quite "simple" (simple is beautiful). This mechanism has proven to perform better/robust than the other open source streaming

[VOTE] FLIP-419: Optimize multi-sink query plan generation

2024-02-21 Thread Jeyhun Karimov
Hi everyone, I'd like to start a vote on the FLIP-419: Optimize multi-sink query plan generation [1]. The discussion thread is here [2]. The vote will be open for at least 72 hours unless there is an objection or insufficient votes. [1]

Re: [ANNOUNCE] New Apache Flink Committer - Jiabao Sun

2024-02-19 Thread Jeyhun Karimov
Congratulations, Jiabao! Well deserved! On Mon, Feb 19, 2024 at 2:26 PM gongzhongqiang wrote: > Congratulations, Jiabao! > > Best, > Zhongqiang Gong > > Qingsheng Ren 于2024年2月19日周一 17:53写道: > > > Hi everyone, > > > > On behalf of the PMC, I'm happy to announce Jiabao Sun as a new Flink > >

[jira] [Created] (FLINK-34459) Results column names should match SELECT clause expression names

2024-02-18 Thread Jeyhun Karimov (Jira)
Jeyhun Karimov created FLINK-34459: -- Summary: Results column names should match SELECT clause expression names Key: FLINK-34459 URL: https://issues.apache.org/jira/browse/FLINK-34459 Project: Flink

[jira] [Created] (FLINK-34442) Support optimizations for pre-partitioned [external] data sources

2024-02-13 Thread Jeyhun Karimov (Jira)
Jeyhun Karimov created FLINK-34442: -- Summary: Support optimizations for pre-partitioned [external] data sources Key: FLINK-34442 URL: https://issues.apache.org/jira/browse/FLINK-34442 Project: Flink

Re: [DISCUSS] FLIP-419: Optimize multi-sink query plan generation

2024-01-30 Thread Jeyhun Karimov
more than welcome. Regards, Jeyhun Karimov On Wed, Jan 17, 2024 at 9:11 PM Jeyhun Karimov wrote: > Hi Jim, > > Thanks for your comments. Please find my answers below: > >1. StreamOptimizeContext may still be needed to pass the fact that we >>are optimizing a

[jira] [Created] (FLINK-34155) Recurring SqlExecutionException

2024-01-18 Thread Jeyhun Karimov (Jira)
Jeyhun Karimov created FLINK-34155: -- Summary: Recurring SqlExecutionException Key: FLINK-34155 URL: https://issues.apache.org/jira/browse/FLINK-34155 Project: Flink Issue Type: Bug

Re: [DISCUSS] FLIP-419: Optimize multi-sink query plan generation

2024-01-17 Thread Jeyhun Karimov
s.apache.org/jira/browse/CALCITE-6188 Seems interesting. But Calcite's MQO approach will probably require some drastic changes in our codebase once we adopt it. This approach is more incremental. Hope my comments answer your questions. Regards, Jeyhun Karimov On Wed, Jan 17, 2024 at 2:36 AM Jim Hugh

[DISCUSS] FLIP-419: Optimize multi-sink query plan generation

2024-01-16 Thread Jeyhun Karimov
]. Looking forward to your feedback. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-419%3A+Optimize+multi-sink+query+plan+generation Regards, Jeyhun Karimov

Re: Permission to add FLIP document

2024-01-15 Thread Jeyhun Karimov
Thank you Leonard. On Mon, Jan 15, 2024 at 2:30 AM Leonard Xu wrote: > Hey, Jeyhun > > I’ve added the permissions for your account. > > Best, > Leonard > > > 2024年1月15日 上午3:42,Jeyhun Karimov 写道: > > > > Dear Flink community, > > > > I would l

Permission to add FLIP document

2024-01-14 Thread Jeyhun Karimov
Dear Flink community, I would like to create a FLIP document. I assume, for that I need to have some permissions (e.g., edit permission for Apache Flink cwiki space). Could you please grant me the required permissions? My cwiki confluence username is jeyhun.karimov Regards, Jeyhun