Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure Management in Kubernetes with Dynamic Termination Log Integration
Hi Swathi C, Also including the Dev mailing list. If you have a good reason for not being able to use the pluggable enricher FLIP, you'll have to include that rationale in your own FLIP and explain it. You might get challenged for it in the Dev mailing list thread discussion, but that's the point. Regards, Martijn On Thu, Apr 25, 2024 at 8:51 AM Swathi C wrote: > Hi Martijn and Ahmed, > > This proposed FLIP was mainly focusing for the CRUD failures use case ( > and not job failures ) and might not be able to use pluggable enricher FLIP > ( as that mainly focuses on job failures ). Hence, for going forward as a > new FLIP, we might not be able to leverage pluggable enricher FLIP for this > use case. So, we might not be able to reformulate it for CRUD failures. > > So, is it ok with this new proposal or let us know if I'm missing > anything and if it is related to pluggable enricher FLIP or anyway we can > use pluggable enricker FLIP here for CRUD failures. > > Regards, > Swathi C > > -- Forwarded message - > From: Martijn Visser > Date: Thu, Apr 25, 2024 at 2:46 AM > Subject: Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure > Management in Kubernetes with Dynamic Termination Log Integration > To: > Cc: , > > > I would prefer a separate FLIP > > On Wed, Apr 24, 2024 at 3:25 PM Swathi C > wrote: > > > Sure Ahmed and Martijn. > > Fetching the flink particular job related failure and adding this logic > to > > termination-log is definitely a sub-task of pluggable enricher as we can > > leverage pluggable enricher to achieve this. > > But for CRUD level failures, which is mainly used to notify if the job > > manager failed might not be using the pluggable enricher. So, let us know > > if that needs to be there as a separate FLIP or we can combine that as > well > > under the pluggable enricher ( by adding another sub task ) ? > > > > Regards, > > Swathi C > > > > On Wed, Apr 24, 2024 at 3:46 PM Ahmed Hamdy > wrote: > > > > > Hi, > > > I agree with the Martijn, We can reformulate the FLIP to introduce > > > termination log as supported pluggable enricher. If you believe the > scope > > > of work is a subset (Further implementation) we can just add a Jira > > ticket > > > for it. IMO this will also help with implementation taking the existing > > > enrichers into reference. > > > Best Regards > > > Ahmed Hamdy > > > > > > > > > On Tue, 23 Apr 2024 at 15:23, Martijn Visser > > > > wrote: > > > > > > > From a procedural point of view, we shouldn't make FLIPs sub-tasks > for > > > > existing FLIPs that have been voted/are released. That will only > cause > > > > confusion down the line. A new FLIP should take existing > functionality > > > > (like FLIP-304) into account, and propose how to improve on what that > > > > original FLIP has introduced or how you're going to leverage what's > > > already > > > > there. > > > > > > > > On Tue, Apr 23, 2024 at 11:42 AM ramkrishna vasudevan < > > > > ramvasu.fl...@gmail.com> wrote: > > > > > > > > > Hi Gyula and Ahmed, > > > > > > > > > > I totally agree that there is an interlap in the final goal that > both > > > the > > > > > FLIPs are achieving here and infact FLIP-304 is more comprehensive > > for > > > > job > > > > > failures. > > > > > > > > > > But as a proposal to move forward can we make Swathi's FLIP/JIRA > as a > > > sub > > > > > task for FLIP-304 and continue with the PR since the main aim is to > > get > > > > the > > > > > cluster failure pushed to the termination log for K8s based > > > deployments. > > > > > And once it is completed we can work to make FLIP-304 to support > job > > > > > failure propagation to termination log? > > > > > > > > > > Regards > > > > > Ram > > > > > > > > > > On Thu, Apr 18, 2024 at 10:07 PM Swathi C < > swathi.c.apa...@gmail.com > > > > > > > > wrote: > > > > > > > > > > > Hi Gyula and Ahmed, > > > > > > > > > > > > Thanks for reviewing this. > > > > > > > > > > > > @gyula.f...@gmail.com , currently since > our > > > aim > > > > > as > > > > > > part of this FLIP was only to fail the cluster when job > > manager/flink > > > > has > > > > > > issues such that the cluster would no longer be usable, hence, we > > > > > proposed > > > > > > only related to that. > > > > > > Your right, that it covers only job main class errors, job > manager > > > run > > > > > time > > > > > > failures, if the Job manager wants to write any metadata to any > > other > > > > > > system ( ABFS, S3 , ... ) and the job failures will not be > > covered. > > > > > > > > > > > > FLIP-304 is mainly used to provide Failure enrichers for job > > > failures. > > > > > > Since, this FLIP is mainly for flink Job manager failures, let us > > > know > > > > if > > > > > > we can leverage the goodness of both and try to extend FLIP-304 > and > > > add > > > > > our > > > > > > plugin implementation to cover the job level issues ( propagate > > this > > > > info > > > > > > to the /dev/termination-log such that, the conta
[jira] [Created] (FLINK-35235) Fix missing dependencies in the uber jar
LvYanquan created FLINK-35235: - Summary: Fix missing dependencies in the uber jar Key: FLINK-35235 URL: https://issues.apache.org/jira/browse/FLINK-35235 Project: Flink Issue Type: Improvement Components: Flink CDC Affects Versions: 3.1.0 Reporter: LvYanquan Fix For: 3.1.0 Attachments: image-2024-04-25-15-17-20-987.png, image-2024-04-25-15-17-34-717.png Some class of Kafka were not included in fat jar. !image-2024-04-25-15-17-34-717.png! -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [DISCUSS] FLIP-445: Support dynamic parallelism inference for HiveSource
Hi Venkat, Thanks for joining the discussion. Based on our understanding, there are still a significant number of existing tasks using Hive. Indeed, many companies are now migrating their data to the lakehouse, but due to historical reasons, a substantial amount of data still resides in Hive. Best, Xia Venkatakrishnan Sowrirajan 于2024年4月25日周四 11:52写道: > Hi Xia, > > +1 on introducing dynamic parallelism inference for HiveSource. > > Orthogonal to this discussion, curious, how commonly HiveSource is used > these days in the industry given the popularity of table formats/sources > like Iceberg, Hudi and Delta lake? > > Thanks > Venkat > > On Wed, Apr 24, 2024, 7:41 PM Xia Sun wrote: > > > Hi everyone, > > > > Thanks for all the feedback! > > > > If there are no more comments, I would like to start the vote thread, > > thanks again! > > > > Best, > > Xia > > > > Ahmed Hamdy 于2024年4月18日周四 21:31写道: > > > > > Hi Xia, > > > I have read through the FLIP and discussion and the new version of the > > FLIP > > > looks better. > > > +1 for the proposal. > > > Best Regards > > > Ahmed Hamdy > > > > > > > > > On Thu, 18 Apr 2024 at 12:21, Ron Liu wrote: > > > > > > > Hi, Xia > > > > > > > > Thanks for updating, looks good to me. > > > > > > > > Best, > > > > Ron > > > > > > > > Xia Sun 于2024年4月18日周四 19:11写道: > > > > > > > > > Hi Ron, > > > > > Yes, presenting it in a table might be more intuitive. I have > already > > > > added > > > > > the table in the "Public Interfaces | New Config Option" chapter of > > > FLIP. > > > > > PTAL~ > > > > > > > > > > Ron Liu 于2024年4月18日周四 18:10写道: > > > > > > > > > > > Hi, Xia > > > > > > > > > > > > Thanks for your reply. > > > > > > > > > > > > > That means, in terms > > > > > > of priority, `table.exec.hive.infer-source-parallelism` > > > > > > > `table.exec.hive.infer-source-parallelism.mode`. > > > > > > > > > > > > I still have some confusion, if the > > > > > > `table.exec.hive.infer-source-parallelism` > > > > > > >`table.exec.hive.infer-source-parallelism.mode`, currently > > > > > > `table.exec.hive.infer-source-parallelism` default value is true, > > > that > > > > > > means always static parallelism inference work? Or perhaps after > > this > > > > > FLIP, > > > > > > we changed the default behavior of > > > > > > `table.exec.hive.infer-source-parallelism` to indicate dynamic > > > > > parallelism > > > > > > inference when enabled. > > > > > > I think you should list the various behaviors of these two > options > > > that > > > > > > coexist in FLIP by a table, only then users can know how the > > dynamic > > > > and > > > > > > static parallelism inference work. > > > > > > > > > > > > Best, > > > > > > Ron > > > > > > > > > > > > Xia Sun 于2024年4月18日周四 16:33写道: > > > > > > > > > > > > > Hi Ron and Lijie, > > > > > > > Thanks for joining the discussion and sharing your suggestions. > > > > > > > > > > > > > > > the InferMode class should also be introduced in the Public > > > > > Interfaces > > > > > > > > section! > > > > > > > > > > > > > > > > > > > > > Thanks for the reminder, I have now added the InferMode class > to > > > the > > > > > > Public > > > > > > > Interfaces section as well. > > > > > > > > > > > > > > > `table.exec.hive.infer-source-parallelism.max` is 1024, I > > checked > > > > > > through > > > > > > > > the code that the default value is 1000? > > > > > > > > > > > > > > > > > > > > > I have checked and the default value of > > > > > > > `table.exec.hive.infer-source-parallelism.max` is indeed 1000. > > This > > > > has > > > > > > > been corrected in the FLIP. > > > > > > > > > > > > > > > how are`table.exec.hive.infer-source-parallelism` and > > > > > > > > `table.exec.hive.infer-source-parallelism.mode` compatible? > > > > > > > > > > > > > > > > > > > > > This is indeed a critical point. The current plan is to > deprecate > > > > > > > `table.exec.hive.infer-source-parallelism` but still utilize it > > as > > > > the > > > > > > main > > > > > > > switch for enabling automatic parallelism inference. That > means, > > in > > > > > terms > > > > > > > of priority, `table.exec.hive.infer-source-parallelism` > > > > > > > > `table.exec.hive.infer-source-parallelism.mode`. In future > > > versions, > > > > if > > > > > > > `table.exec.hive.infer-source-parallelism` is removed, this > logic > > > > will > > > > > > also > > > > > > > need to be revised, leaving only > > > > > > > `table.exec.hive.infer-source-parallelism.mode` as the basis > for > > > > > deciding > > > > > > > whether to enable parallelism inference. I have also added this > > > > > > description > > > > > > > to the FLIP. > > > > > > > > > > > > > > > > > > > > > > In FLIP-367 it is supported to be able to set the Source's > > > > > parallelism > > > > > > > > individually, if in the future HiveSource also supports this > > > > feature, > > > > > > > > however, the default value of > > > > > > > > `table.exec.hive.infer-source-parallelism.mode` is > > > > > `InferMode.DYNAMI
[DISCUSSION] FLIP-450: Improve Runtime Configuration for Flink 2.0
Hi all, I'd like to start a discussion on FLIP-450: Improve Runtime Configuration for Flink 2.0 [1]. As Flink moves toward 2.0, we have revisited all runtime configurations and identified several improvements to enhance user-friendliness and maintainability. In this FLIP, we aim to refine the runtime configuration. Looking forward to everyone's feedback and suggestions. Thank you! Best regards, Xuannan [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-450%3A+Improve+Runtime+Configuration+for+Flink+2.0
[jira] [Created] (FLINK-35236) Flink 1.19 Translation error on the Chinese official website
hongxu han created FLINK-35236: -- Summary: Flink 1.19 Translation error on the Chinese official website Key: FLINK-35236 URL: https://issues.apache.org/jira/browse/FLINK-35236 Project: Flink Issue Type: Bug Components: chinese-translation Affects Versions: 1.19.0 Reporter: hongxu han [https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/datastream/execution_mode/#order-of-processing] -- This message was sent by Atlassian Jira (v8.20.10#820010)
[DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc
Hi all, I'd like to start a discussion on FLIP-449: Reorganization of flink-connector-jdbc [1]. As Flink continues to evolve, we've noticed an increasing level of complexity within the JDBC connector. The proposed solution is to address this complexity by separating the core functionality from individual database components, thereby streamlining the structure into distinct modules. Looking forward to your feedback and suggestions, thanks. Best regards, Joao Boto [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-449%3A+Reorganization+of+flink-connector-jdbc
[jira] [Created] (FLINK-35237) Allow Custom HashFunction in PrePartitionOperator for Flink Sink Customization
zhangdingxin created FLINK-35237: Summary: Allow Custom HashFunction in PrePartitionOperator for Flink Sink Customization Key: FLINK-35237 URL: https://issues.apache.org/jira/browse/FLINK-35237 Project: Flink Issue Type: Improvement Components: Flink CDC Reporter: zhangdingxin The {{PrePartitionOperator}} in its current implementation only supports a fixed {{HashFunction}} ({{{}org.apache.flink.cdc.runtime.partitioning.PrePartitionOperator.HashFunction{}}}). This limits the ability of Sink implementations to customize the partitioning logic for {{{}DataChangeEvent{}}}s. For example, in the case of partitioned tables, it would be advantageous to allow hashing based on partition keys, hashing according to table names, or using the database engine's internal primary key hash functions (such as with MaxCompute DataSink). When users require such custom partitioning logic, they are compelled to implement their PartitionOperator, which undermines the utility of {{{}PrePartitionOperator{}}}. To address this limitation, it would be highly desirable to enable the {{PrePartitionOperator}} to support user-specified custom {{{}HashFunction{}}}s (Function). A possible solution could involve a mechanism analogous to the {{DataSink}} interface, allowing the specification of a {{HashFunctionFactory}} class path in the configuration file. This enhancement would greatly facilitate users in tailoring partition strategies to meet their specific application needs. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (FLINK-35238) AZP is not working since some ci agent unhealthy
Weijie Guo created FLINK-35238: -- Summary: AZP is not working since some ci agent unhealthy Key: FLINK-35238 URL: https://issues.apache.org/jira/browse/FLINK-35238 Project: Flink Issue Type: Bug Components: Build System / CI Reporter: Weijie Guo -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table
Hi, Lorenzo and Feng Thanks for joining this thread discussing. Sorry for later response, regarding your question: > About the Operations interfaces, how can they be empty? Should not they provide at least a `run` or `execute` method (similar to the command pattern)? In this way, their implementation can wrap all the implementations details of particular schedulers, and the scheduler can simply execute the command. In general, I think a simple sequence diagram showcasing the interaction between the interfaces would be awesome to better understand the concept. I've updated the FLIP, and added the Outline Design section, to introduce how Materialized Table interacts with the Workflow Scheduler in Full Refresh mode via a timing diagram, it can help to understand this proposal design. > What about the RefreshHandler, I cannot find a definition of its interface here. Is it out of scope for this FLIP? There is some context that is not aligned here, RefreshHandler was proposed in FLIP-435, you can get more detail from [1]. > If it is periodic, where is the period? For the scheduleTime and format, why not simply pass an instance of LocalDateTime or similar? The gateway should not have the responsibility to parse the time. This might require a bit of context for clarity. In Full Refresh mode, the Materialized Table requires the Scheduler to periodically trigger refresh operations. This means that the Scheduler will periodically call the REST API, passing parameters such as scheduleTime and scheduleTimeFormat. The materialized table manager(to be introduced) relies on this information to accurately calculate the correct time partitions. At the same time, we also support manual refreshes of materialized tables, and in the future, we will support manual cascading refreshes on a multi-table granularity. For cases of manual cascading refresh, we will also register a one-time refresh workflow with the Scheduler, which then triggers the execution via the REST API call. However, during a manual refresh, users typically specify partition information, and there's no need for the engine to deduce it, thus scheduler time is not needed. Taking the above into account, there are two types of refresh workflows: periodic workflows and one-time workflows. The engine requires different information for each type of workflow. When designing the REST API, we aim for this API to support both types of workflows simultaneously. Hence we introduce the isPeriodic parameter for differentiation. Then the engine will know what to do accordingly. The scheduleTime and scheduleTimeFormat are passed from Scheduler to the Gateway via the REST API. Firstly, in the HTTP protocol, there is no type equivalent to Java's LocalDateTime. Secondly, Schedulers can potentially be written in different programming languages; for example, Airflow uses Python to develop its workflows. Hence, it's obvious that we cannot limit the Scheduler to the use of Java LocalDateTime type. Therefore, a String type is the most suitable. Lastly, the purpose of the schedulerTime is to determine the time partitioning details of the partitioned table. This parsing responsibility falls upon the materialized table manager and not the SqlGateway, which is solely responsible for passthrough parameters. You may refer to the Outline Design section of this FLIP, specifically the Partitioned Table Full Refresh part in FLIP-435, to further comprehend the overall design principles. > For the REST API: wouldn't it be better (more REST) to move the `mt_identifier` to the URL? E.g.: v3/materialized_tables//refresh I think this is a good idea. I have another consideration though, does this API support passing multiple materialized tables at the same time, if it does, it will have to be put in the request body. I will discuss the design of this API with ShengKai Fang offline, he is the owner of the Gateway module. Anyway, your proposal is a good choice. > From my current understanding, the workflow handle should not be bound to the Dynamic Table. Therefore, if the workflow is modified, does it mean that the scheduling information corresponding to the Dynamic Table will be lost? You can see the FLIP Outline Design section to understand the overall design further. The refresh handler is just a pointer that can locate the workflow info in the scheduler, so scheduling info will be persistent to the Scheduler, it will not lost. > Regarding the status information of the workflow, I am wondering if it is necessary to provide an interface to display the backend scheduling information? This would make it more convenient to view the execution status of backend jobs. The RefreshHandler#asSummaryString will return the summary information of the background refresh job, you can get it via DESC TABLE xxx. I think you want to get detail information about background jobs, you should go to Scheduler, it provides the most detailed information. Even if the interface is provided, we don't get th
Re:[DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc
Hi, Boto. Thanks for your driving it ! +1 from me on the proposal. Maybe we need to ensure that a simple usage method is provided to users after the refactoring. In the current situation, which supported database does the user intend to use, Users only need to add the flink-connector-jdbc-${version}.jar + ${database-connector-driver}.jar into the dependencies, which could be used out of the box. I noticed in FLIP that we will perform shadow related operations to ensure the same usage and semantics as before. So, if users only want to use one type of database (eg. MySQL), what forms would we plan to provide jars in? For example: 1. flink-connector-jdbc-core-${version}.jar + flink-connector-jdbc-mysql-${version}.jar + ${database-connector-driver}.jar. 2. Or flink-connector-jdbc-mysql-${version}.jar + ${database-connector-driver}.jar. 3. Or a another different concise way? Thank you. Best, Yuepeng Pan At 2024-04-25 16:54:13, "Joao Boto" wrote: >Hi all, > >I'd like to start a discussion on FLIP-449: Reorganization of >flink-connector-jdbc [1]. >As Flink continues to evolve, we've noticed an increasing level of >complexity within the JDBC connector. >The proposed solution is to address this complexity by separating the core >functionality from individual database components, thereby streamlining the >structure into distinct modules. > >Looking forward to your feedback and suggestions, thanks. >Best regards, >Joao Boto > >[1] >https://cwiki.apache.org/confluence/display/FLINK/FLIP-449%3A+Reorganization+of+flink-connector-jdbc
[jira] [Created] (FLINK-35239) 1.19 docs show outdated warning
Ufuk Celebi created FLINK-35239: --- Summary: 1.19 docs show outdated warning Key: FLINK-35239 URL: https://issues.apache.org/jira/browse/FLINK-35239 Project: Flink Issue Type: Bug Components: Documentation Affects Versions: 1.19.0 Reporter: Ufuk Celebi Assignee: Ufuk Celebi Fix For: 1.19.0 Attachments: Screenshot 2024-04-25 at 15.01.57.png The docs for 1.19 are currently marked as outdated although it's the currently stable release. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD
+1 (binding) On Wed, Apr 24, 2024 at 5:14 AM Yuepeng Pan wrote: > +1(non-binding) > > > Best, > Yuepeng Pan > > At 2024-04-24 16:05:07, "Rui Fan" <1996fan...@gmail.com> wrote: > >+1(binding) > > > >Best, > >Rui > > > >On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany wrote: > > > >> Hi everyone, > >> > >> I'd like to start a vote on the FLIP-446: Kubernetes Operator State > >> Snapshot CRD [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] > >> > >> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD > >> [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7 > >> > >> Regards, > >> Mate > >> >
Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure Management in Kubernetes with Dynamic Termination Log Integration
Thanks everyone for the feedback. Will try to dive deep into Pluggable Enrichers to see how we can incorporate termination-log by using it. Regards, Swathi C On Thu, Apr 25, 2024 at 12:33 PM Martijn Visser wrote: > Hi Swathi C, > > Also including the Dev mailing list. > > If you have a good reason for not being able to use the pluggable enricher > FLIP, you'll have to include that rationale in your own FLIP and explain > it. You might get challenged for it in the Dev mailing list thread > discussion, but that's the point. > > Regards, > > Martijn > > On Thu, Apr 25, 2024 at 8:51 AM Swathi C > wrote: > >> Hi Martijn and Ahmed, >> >> This proposed FLIP was mainly focusing for the CRUD failures use case ( >> and not job failures ) and might not be able to use pluggable enricher FLIP >> ( as that mainly focuses on job failures ). Hence, for going forward as a >> new FLIP, we might not be able to leverage pluggable enricher FLIP for this >> use case. So, we might not be able to reformulate it for CRUD failures. >> >> So, is it ok with this new proposal or let us know if I'm missing >> anything and if it is related to pluggable enricher FLIP or anyway we can >> use pluggable enricker FLIP here for CRUD failures. >> >> Regards, >> Swathi C >> >> -- Forwarded message - >> From: Martijn Visser >> Date: Thu, Apr 25, 2024 at 2:46 AM >> Subject: Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure >> Management in Kubernetes with Dynamic Termination Log Integration >> To: >> Cc: , >> >> >> I would prefer a separate FLIP >> >> On Wed, Apr 24, 2024 at 3:25 PM Swathi C >> wrote: >> >> > Sure Ahmed and Martijn. >> > Fetching the flink particular job related failure and adding this logic >> to >> > termination-log is definitely a sub-task of pluggable enricher as we can >> > leverage pluggable enricher to achieve this. >> > But for CRUD level failures, which is mainly used to notify if the job >> > manager failed might not be using the pluggable enricher. So, let us >> know >> > if that needs to be there as a separate FLIP or we can combine that as >> well >> > under the pluggable enricher ( by adding another sub task ) ? >> > >> > Regards, >> > Swathi C >> > >> > On Wed, Apr 24, 2024 at 3:46 PM Ahmed Hamdy >> wrote: >> > >> > > Hi, >> > > I agree with the Martijn, We can reformulate the FLIP to introduce >> > > termination log as supported pluggable enricher. If you believe the >> scope >> > > of work is a subset (Further implementation) we can just add a Jira >> > ticket >> > > for it. IMO this will also help with implementation taking the >> existing >> > > enrichers into reference. >> > > Best Regards >> > > Ahmed Hamdy >> > > >> > > >> > > On Tue, 23 Apr 2024 at 15:23, Martijn Visser < >> martijnvis...@apache.org> >> > > wrote: >> > > >> > > > From a procedural point of view, we shouldn't make FLIPs sub-tasks >> for >> > > > existing FLIPs that have been voted/are released. That will only >> cause >> > > > confusion down the line. A new FLIP should take existing >> functionality >> > > > (like FLIP-304) into account, and propose how to improve on what >> that >> > > > original FLIP has introduced or how you're going to leverage what's >> > > already >> > > > there. >> > > > >> > > > On Tue, Apr 23, 2024 at 11:42 AM ramkrishna vasudevan < >> > > > ramvasu.fl...@gmail.com> wrote: >> > > > >> > > > > Hi Gyula and Ahmed, >> > > > > >> > > > > I totally agree that there is an interlap in the final goal that >> both >> > > the >> > > > > FLIPs are achieving here and infact FLIP-304 is more comprehensive >> > for >> > > > job >> > > > > failures. >> > > > > >> > > > > But as a proposal to move forward can we make Swathi's FLIP/JIRA >> as a >> > > sub >> > > > > task for FLIP-304 and continue with the PR since the main aim is >> to >> > get >> > > > the >> > > > > cluster failure pushed to the termination log for K8s based >> > > deployments. >> > > > > And once it is completed we can work to make FLIP-304 to support >> job >> > > > > failure propagation to termination log? >> > > > > >> > > > > Regards >> > > > > Ram >> > > > > >> > > > > On Thu, Apr 18, 2024 at 10:07 PM Swathi C < >> swathi.c.apa...@gmail.com >> > > >> > > > > wrote: >> > > > > >> > > > > > Hi Gyula and Ahmed, >> > > > > > >> > > > > > Thanks for reviewing this. >> > > > > > >> > > > > > @gyula.f...@gmail.com , currently since >> our >> > > aim >> > > > > as >> > > > > > part of this FLIP was only to fail the cluster when job >> > manager/flink >> > > > has >> > > > > > issues such that the cluster would no longer be usable, hence, >> we >> > > > > proposed >> > > > > > only related to that. >> > > > > > Your right, that it covers only job main class errors, job >> manager >> > > run >> > > > > time >> > > > > > failures, if the Job manager wants to write any metadata to any >> > other >> > > > > > system ( ABFS, S3 , ... ) and the job failures will not be >> > covered. >> > > > > > >> > > > > > FLIP-304 is mainly used t
Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD
In principle I'm +1 on the proposal, but I think the FLIP in the wiki is not in sync with the Google doc. For example in the Wiki FlinkStateSnapshotSpec.backoffLimit is missing. On Thu, Apr 25, 2024 at 3:27 PM Thomas Weise wrote: > +1 (binding) > > > On Wed, Apr 24, 2024 at 5:14 AM Yuepeng Pan wrote: > > > +1(non-binding) > > > > > > Best, > > Yuepeng Pan > > > > At 2024-04-24 16:05:07, "Rui Fan" <1996fan...@gmail.com> wrote: > > >+1(binding) > > > > > >Best, > > >Rui > > > > > >On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany > wrote: > > > > > >> Hi everyone, > > >> > > >> I'd like to start a vote on the FLIP-446: Kubernetes Operator State > > >> Snapshot CRD [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] > > >> > > >> > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD > > >> [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7 > > >> > > >> Regards, > > >> Mate > > >> > > >
Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD
That's my fault @Robert Metzger , since the new FLIP process and a lack of confluent access for non-committers this is a bit tricky to get it sync :) Gyula On Thu, Apr 25, 2024 at 4:17 PM Robert Metzger wrote: > In principle I'm +1 on the proposal, but I think the FLIP in the wiki is > not in sync with the Google doc. > For example in the Wiki FlinkStateSnapshotSpec.backoffLimit is missing. > > On Thu, Apr 25, 2024 at 3:27 PM Thomas Weise wrote: > > > +1 (binding) > > > > > > On Wed, Apr 24, 2024 at 5:14 AM Yuepeng Pan > wrote: > > > > > +1(non-binding) > > > > > > > > > Best, > > > Yuepeng Pan > > > > > > At 2024-04-24 16:05:07, "Rui Fan" <1996fan...@gmail.com> wrote: > > > >+1(binding) > > > > > > > >Best, > > > >Rui > > > > > > > >On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany > > wrote: > > > > > > > >> Hi everyone, > > > >> > > > >> I'd like to start a vote on the FLIP-446: Kubernetes Operator State > > > >> Snapshot CRD [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] > > > >> > > > >> > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD > > > >> [2] > https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7 > > > >> > > > >> Regards, > > > >> Mate > > > >> > > > > > >
Re: Re:[DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc
Hi Pan, Users who wish to utilize only one database and prefer not to use flink-connector-jdbc-${version}.jar + ${database-connector-driver}.jar should opt for option 1: flink-connector-jdbc-core-${version}.jar + flink-connector-jdbc-mysql-${version}.jar + ${database-connector-driver}.jar. We could introduce a flink-connector-jdbc-mysql-${version}-fat.jar that includes flink-connector-jdbc-core-${version}.jar, but this could create potential challenges. This approach could lead to duplicate classes if a user intends to read from MySQL and write to PostgreSQL while utilizing both fat JARs simultaneously. To maintain clarity and minimize conflicts, we're currently leaning towards maintaining the existing structure, where flink-connector-jdbc-${version}.jar remains shaded for simplicity, encompassing the core functionality and all database-related features within the same JAR. Please let me know if you require further clarification on any aspect. Best regards, Joao Boto On 2024/04/25 11:41:00 Yuepeng Pan wrote: > Hi, Boto. > > Thanks for your driving it ! > +1 from me on the proposal. > > > > > Maybe we need to ensure that a simple usage method is provided to users after > the refactoring. > > In the current situation, which supported database does the user intend to > use, > > Users only need to add the flink-connector-jdbc-${version}.jar + > ${database-connector-driver}.jar > > into the dependencies, which could be used out of the box. > > I noticed in FLIP that we will perform shadow related operations to ensure > > the same usage and semantics as before. > > So, if users only want to use one type of database (eg. MySQL), > > what forms would we plan to provide jars in? > > For example: > > 1. flink-connector-jdbc-core-${version}.jar + > flink-connector-jdbc-mysql-${version}.jar + ${database-connector-driver}.jar. > > 2. Or flink-connector-jdbc-mysql-${version}.jar + > ${database-connector-driver}.jar. > > 3. Or a another different concise way? > > > > > Thank you. > > Best, > Yuepeng Pan > > > > > At 2024-04-25 16:54:13, "Joao Boto" wrote: > >Hi all, > > > >I'd like to start a discussion on FLIP-449: Reorganization of > >flink-connector-jdbc [1]. > >As Flink continues to evolve, we've noticed an increasing level of > >complexity within the JDBC connector. > >The proposed solution is to address this complexity by separating the core > >functionality from individual database components, thereby streamlining the > >structure into distinct modules. > > > >Looking forward to your feedback and suggestions, thanks. > >Best regards, > >Joao Boto > > > >[1] > >https://cwiki.apache.org/confluence/display/FLINK/FLIP-449%3A+Reorganization+of+flink-connector-jdbc >
Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD
Ah, true -- now I remember. Thanks for fixing the wiki page +1 (binding) On Thu, Apr 25, 2024 at 4:40 PM Gyula Fóra wrote: > That's my fault @Robert Metzger , since the new FLIP > process and a lack of confluent access for non-committers this is a bit > tricky to get it sync :) > > Gyula > > On Thu, Apr 25, 2024 at 4:17 PM Robert Metzger > wrote: > > > In principle I'm +1 on the proposal, but I think the FLIP in the wiki is > > not in sync with the Google doc. > > For example in the Wiki FlinkStateSnapshotSpec.backoffLimit is missing. > > > > On Thu, Apr 25, 2024 at 3:27 PM Thomas Weise wrote: > > > > > +1 (binding) > > > > > > > > > On Wed, Apr 24, 2024 at 5:14 AM Yuepeng Pan > > wrote: > > > > > > > +1(non-binding) > > > > > > > > > > > > Best, > > > > Yuepeng Pan > > > > > > > > At 2024-04-24 16:05:07, "Rui Fan" <1996fan...@gmail.com> wrote: > > > > >+1(binding) > > > > > > > > > >Best, > > > > >Rui > > > > > > > > > >On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany > > > wrote: > > > > > > > > > >> Hi everyone, > > > > >> > > > > >> I'd like to start a vote on the FLIP-446: Kubernetes Operator > State > > > > >> Snapshot CRD [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] > > > > >> > > > > >> > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD > > > > >> [2] > > https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7 > > > > >> > > > > >> Regards, > > > > >> Mate > > > > >> > > > > > > > > > >
[jira] [Created] (FLINK-35240) Disable FLUSH_AFTER_WRITE_VALUE to avoid flush per record
Zhongqiang Gong created FLINK-35240: --- Summary: Disable FLUSH_AFTER_WRITE_VALUE to avoid flush per record Key: FLINK-35240 URL: https://issues.apache.org/jira/browse/FLINK-35240 Project: Flink Issue Type: Improvement Components: Connectors / FileSystem Reporter: Zhongqiang Gong Attachments: image-2024-04-26-00-23-29-975.png According to user email: https://lists.apache.org/thread/9j5z8hv4vjkd54dkzqy1ryyvm0l5rxhc !image-2024-04-26-00-23-29-975.png! I found `org.apache.flink.formats.csv.CsvBulkWriter#addElement` will flush per record. I think maybe we can disable `FLUSH_AFTER_WRITE_VALUE` to avoid flush when a record added. -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD
+1( (non-binding) Best, Zhongqiang Gong Mate Czagany 于2024年4月24日周三 16:06写道: > Hi everyone, > > I'd like to start a vote on the FLIP-446: Kubernetes Operator State > Snapshot CRD [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] > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD > [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7 > > Regards, > Mate >
[jira] [Created] (FLINK-35241) Support SQL FLOOR and CEIL functions with SECOND and MINUTE for TIMESTAMP_TLZ
Alexey Leonov-Vendrovskiy created FLINK-35241: - Summary: Support SQL FLOOR and CEIL functions with SECOND and MINUTE for TIMESTAMP_TLZ Key: FLINK-35241 URL: https://issues.apache.org/jira/browse/FLINK-35241 Project: Flink Issue Type: Bug Components: Table SQL / API Reporter: Alexey Leonov-Vendrovskiy We need a fix for both SECOND and MINUTE. The following query doesn't work: {code:java} SELECT FLOOR( CAST(TIMESTAMP '2024-04-25 17:19:42.654' AS TIMESTAMP_LTZ(3)) TO MINUTE) {code} These two queries work: {code:java} SELECT FLOOR( CAST(TIMESTAMP '2024-04-25 17:19:42.654' AS TIMESTAMP_LTZ(3)) TO HOUR) {code} {code:java} SELECT FLOOR( TIMESTAMP '2024-04-25 17:19:42.654' TO MINUTE) {code} Stack trace: For the first not working query: {code:java} Caused by: io.confluent.flink.table.utils.CleanedException: org.codehaus.commons.compiler.CompileException: Line 41, Column 69: No applicable constructor/method found for actual parameters "org.apache.flink.table.data.TimestampData, org.apache.flink.table.data.TimestampData"; candidates are: "public static long org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(long, long)", "public static float org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(float)", "public static org.apache.flink.table.data.DecimalData org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(org.apache.flink.table.data.DecimalData)", "public static int org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(int, int)", "public static double org.apache.flink.table.runtime.functions.SqlFunctionUtils.floor(double)" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:13080) at org.codehaus.janino.UnitCompiler.findMostSpecificIInvocable(UnitCompiler.java:9646) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9506) at org.codehaus.janino.UnitCompiler.findIMethod(UnitCompiler.java:9422) at org.codehaus.janino.UnitCompiler.compileGet2(UnitCompiler.java:5263) ... {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
Re: Re:[DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc
Hi, Boto. It's already clear enough to me. Thanks for your reply. Best, Yuepeng Pan On 2024/04/25 15:41:01 João Boto wrote: > Hi Pan, > > Users who wish to utilize only one database and prefer not to use > flink-connector-jdbc-${version}.jar + ${database-connector-driver}.jar should > opt for option 1: flink-connector-jdbc-core-${version}.jar + > flink-connector-jdbc-mysql-${version}.jar + ${database-connector-driver}.jar. > > We could introduce a flink-connector-jdbc-mysql-${version}-fat.jar that > includes flink-connector-jdbc-core-${version}.jar, but this could create > potential challenges. This approach could lead to duplicate classes if a user > intends to read from MySQL and write to PostgreSQL while utilizing both fat > JARs simultaneously. > > To maintain clarity and minimize conflicts, we're currently leaning towards > maintaining the existing structure, where flink-connector-jdbc-${version}.jar > remains shaded for simplicity, encompassing the core functionality and all > database-related features within the same JAR. > > Please let me know if you require further clarification on any aspect. > > Best regards, > Joao Boto > > > > On 2024/04/25 11:41:00 Yuepeng Pan wrote: > > Hi, Boto. > > > > Thanks for your driving it ! > > +1 from me on the proposal. > > > > > > > > > > Maybe we need to ensure that a simple usage method is provided to users > > after the refactoring. > > > > In the current situation, which supported database does the user intend to > > use, > > > > Users only need to add the flink-connector-jdbc-${version}.jar + > > ${database-connector-driver}.jar > > > > into the dependencies, which could be used out of the box. > > > > I noticed in FLIP that we will perform shadow related operations to ensure > > > > the same usage and semantics as before. > > > > So, if users only want to use one type of database (eg. MySQL), > > > > what forms would we plan to provide jars in? > > > > For example: > > > > 1. flink-connector-jdbc-core-${version}.jar + > > flink-connector-jdbc-mysql-${version}.jar + > > ${database-connector-driver}.jar. > > > > 2. Or flink-connector-jdbc-mysql-${version}.jar + > > ${database-connector-driver}.jar. > > > > 3. Or a another different concise way? > > > > > > > > > > Thank you. > > > > Best, > > Yuepeng Pan > > > > > > > > > > At 2024-04-25 16:54:13, "Joao Boto" wrote: > > >Hi all, > > > > > >I'd like to start a discussion on FLIP-449: Reorganization of > > >flink-connector-jdbc [1]. > > >As Flink continues to evolve, we've noticed an increasing level of > > >complexity within the JDBC connector. > > >The proposed solution is to address this complexity by separating the core > > >functionality from individual database components, thereby streamlining the > > >structure into distinct modules. > > > > > >Looking forward to your feedback and suggestions, thanks. > > >Best regards, > > >Joao Boto > > > > > >[1] > > >https://cwiki.apache.org/confluence/display/FLINK/FLIP-449%3A+Reorganization+of+flink-connector-jdbc > > >
Re: [DISCUSSION] FLIP-449: Reorganization of flink-connector-jdbc
Hey João, Thanks for FLIP proposal! Since proposal is to introduce modules, would it make sense to have another module for APIs (flink-jdbc-connector-api)? For this I would suggest to move all public interfaces (e.g, JdbcRowConverter, JdbcConnectionProvider). And even convert some classes into interface with their default implementations, for example, JdbcSink, JdbcConnectionOptions. This way users would have clear interfaces to build their own JDBC based Flink connectors. Here I am not suggesting to introduce new interfaces, only suggest also to separate the API from the core implementation. What do you think? Best, Muhammet On 2024-04-25 08:54, Joao Boto wrote: Hi all, I'd like to start a discussion on FLIP-449: Reorganization of flink-connector-jdbc [1]. As Flink continues to evolve, we've noticed an increasing level of complexity within the JDBC connector. The proposed solution is to address this complexity by separating the core functionality from individual database components, thereby streamlining the structure into distinct modules. Looking forward to your feedback and suggestions, thanks. Best regards, Joao Boto [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-449%3A+Reorganization+of+flink-connector-jdbc
[VOTE] FLIP-445: Support dynamic parallelism inference for HiveSource
Hi everyone, I'd like to start a vote on FLIP-445: Support dynamic parallelism inference for HiveSource[1] which has been discussed in this thread [2]. The vote will be open for at least 72 hours unless there is an objection or not enough votes. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-445%3A+Support+dynamic+parallelism+inference+for+HiveSource [2] https://lists.apache.org/thread/4k1qx6lodhbkknkhjyl0lq9bx8fcpjvn Best, Xia
Re: [VOTE] FLIP-445: Support dynamic parallelism inference for HiveSource
Hey Xia, +1 (non-binding) Thanks and best, Muhammet On 2024-04-26 02:21, Xia Sun wrote: Hi everyone, I'd like to start a vote on FLIP-445: Support dynamic parallelism inference for HiveSource[1] which has been discussed in this thread [2]. The vote will be open for at least 72 hours unless there is an objection or not enough votes. [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-445%3A+Support+dynamic+parallelism+inference+for+HiveSource [2] https://lists.apache.org/thread/4k1qx6lodhbkknkhjyl0lq9bx8fcpjvn Best, Xia
Re: [VOTE] FLIP-445: Support dynamic parallelism inference for HiveSource
+1(binding) Best, Rui On Fri, Apr 26, 2024 at 10:26 AM Muhammet Orazov wrote: > Hey Xia, > > +1 (non-binding) > > Thanks and best, > Muhammet > > On 2024-04-26 02:21, Xia Sun wrote: > > Hi everyone, > > > > I'd like to start a vote on FLIP-445: Support dynamic parallelism > > inference > > for HiveSource[1] which has been discussed in this thread [2]. > > > > The vote will be open for at least 72 hours unless there is an > > objection or > > not enough votes. > > > > > > [1] > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-445%3A+Support+dynamic+parallelism+inference+for+HiveSource > > [2] https://lists.apache.org/thread/4k1qx6lodhbkknkhjyl0lq9bx8fcpjvn > > > > > > Best, > > Xia >
Re: [VOTE] FLIP-445: Support dynamic parallelism inference for HiveSource
+1(binding) Best, Ron Rui Fan <1996fan...@gmail.com> 于2024年4月26日周五 12:55写道: > +1(binding) > > Best, > Rui > > On Fri, Apr 26, 2024 at 10:26 AM Muhammet Orazov > wrote: > > > Hey Xia, > > > > +1 (non-binding) > > > > Thanks and best, > > Muhammet > > > > On 2024-04-26 02:21, Xia Sun wrote: > > > Hi everyone, > > > > > > I'd like to start a vote on FLIP-445: Support dynamic parallelism > > > inference > > > for HiveSource[1] which has been discussed in this thread [2]. > > > > > > The vote will be open for at least 72 hours unless there is an > > > objection or > > > not enough votes. > > > > > > > > > [1] > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-445%3A+Support+dynamic+parallelism+inference+for+HiveSource > > > [2] https://lists.apache.org/thread/4k1qx6lodhbkknkhjyl0lq9bx8fcpjvn > > > > > > > > > Best, > > > Xia > > >
Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax
Hi Yubin, Thanks for your effort. +1 with the display layout change (binding). Best, Jane On Wed, Apr 24, 2024 at 5:28 PM Ahmed Hamdy wrote: > Hi, +1 (non-binding) > Best Regards > Ahmed Hamdy > > > On Wed, 24 Apr 2024 at 09:58, Yubin Li wrote: > > > Hi everyone, > > > > During the implementation of the "describe catalog" syntax, it was > > found that the original output style needed to be improved. > > ``` > > desc catalog extended cat2; > > > > > +--+-+ > > | catalog_description_item | > > catalog_description_value | > > > > > +--+-+ > > | Name | > > cat2 | > > | Type | > > generic_in_memory | > > | Comment | > > | > > | Properties | ('default-database','db'), > > ('type','generic_in_memory') | > > > > > +--+-+ > > 4 rows in set > > ``` > > After offline discussions with Jane Chan and Jark Wu, we suggest > > improving it to the following form: > > ``` > > desc catalog extended cat2; > > +-+---+ > > | info name |info value | > > +-+---+ > > |name | cat2 | > > |type | generic_in_memory | > > | comment | | > > | option:default-database |db | > > +-+---+ > > 4 rows in set > > ``` > > > > For the following reasons: > > 1. The title should be consistent with engines such as Databricks for > > easy understanding, and it should also be consistent with Flink's own > > naming style. Therefore, the title adopts "info name", "info value", > > and the key name should be unified in lowercase, so "Name" is replaced > > by "name". > > Note: Databricks output style [1] as follows: > > ``` > > > DESCRIBE CATALOG main; > > info_name info_value > > > > Catalog Name main > > Comment Main catalog (auto-created) > > Owner metastore-admin-users > > Catalog Type Regular > > ``` > > 2. There may be many attributes of the catalog, and it is very poor in > > readability when displayed in one line. It should be expanded into > > multiple lines, and the key name is prefixed with "option:" to > > identify that this is an attribute row. And since `type` is an > > important information of the catalog, even if `extended` is not > > specified, it should also be displayed, and correspondingly, > > "option:type" should be removed to avoid redundancy. > > > > WDYT? Looking forward to your reply! > > > > [1] > > > https://learn.microsoft.com/zh-tw/azure/databricks/sql/language-manual/sql-ref-syntax-aux-describe-catalog > > > > Best, > > Yubin > > > > On Wed, Mar 20, 2024 at 2:15 PM Benchao Li wrote: > > > > > > +1 (binding) > > > > > > gongzhongqiang 于2024年3月20日周三 11:40写道: > > > > > > > > +1 (non-binding) > > > > > > > > Best, > > > > Zhongqiang Gong > > > > > > > > Yubin Li 于2024年3月19日周二 18:03写道: > > > > > > > > > Hi everyone, > > > > > > > > > > Thanks for all the feedback, I'd like to start a vote on the > > FLIP-436: > > > > > Introduce Catalog-related Syntax [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] > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-436%3A+Introduce+Catalog-related+Syntax > > > > > [2] > https://lists.apache.org/thread/10k1bjb4sngyjwhmfqfky28lyoo7sv0z > > > > > > > > > > Best regards, > > > > > Yubin > > > > > > > > > > > > > > > > > -- > > > > > > Best, > > > Benchao Li > > >
Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax
Thanks for driving this, Jane and Yubin. +1. The new layout looks good to me. Best, Jark On Fri, 26 Apr 2024 at 13:57, Jane Chan wrote: > Hi Yubin, > > Thanks for your effort. +1 with the display layout change (binding). > > Best, > Jane > > On Wed, Apr 24, 2024 at 5:28 PM Ahmed Hamdy wrote: > > > Hi, +1 (non-binding) > > Best Regards > > Ahmed Hamdy > > > > > > On Wed, 24 Apr 2024 at 09:58, Yubin Li wrote: > > > > > Hi everyone, > > > > > > During the implementation of the "describe catalog" syntax, it was > > > found that the original output style needed to be improved. > > > ``` > > > desc catalog extended cat2; > > > > > > > > > +--+-+ > > > | catalog_description_item | > > > catalog_description_value | > > > > > > > > > +--+-+ > > > | Name | > > > cat2 | > > > | Type | > > > generic_in_memory | > > > | Comment | > > > | > > > | Properties | ('default-database','db'), > > > ('type','generic_in_memory') | > > > > > > > > > +--+-+ > > > 4 rows in set > > > ``` > > > After offline discussions with Jane Chan and Jark Wu, we suggest > > > improving it to the following form: > > > ``` > > > desc catalog extended cat2; > > > +-+---+ > > > | info name |info value | > > > +-+---+ > > > |name | cat2 | > > > |type | generic_in_memory | > > > | comment | | > > > | option:default-database |db | > > > +-+---+ > > > 4 rows in set > > > ``` > > > > > > For the following reasons: > > > 1. The title should be consistent with engines such as Databricks for > > > easy understanding, and it should also be consistent with Flink's own > > > naming style. Therefore, the title adopts "info name", "info value", > > > and the key name should be unified in lowercase, so "Name" is replaced > > > by "name". > > > Note: Databricks output style [1] as follows: > > > ``` > > > > DESCRIBE CATALOG main; > > > info_name info_value > > > > > > Catalog Name main > > > Comment Main catalog (auto-created) > > > Owner metastore-admin-users > > > Catalog Type Regular > > > ``` > > > 2. There may be many attributes of the catalog, and it is very poor in > > > readability when displayed in one line. It should be expanded into > > > multiple lines, and the key name is prefixed with "option:" to > > > identify that this is an attribute row. And since `type` is an > > > important information of the catalog, even if `extended` is not > > > specified, it should also be displayed, and correspondingly, > > > "option:type" should be removed to avoid redundancy. > > > > > > WDYT? Looking forward to your reply! > > > > > > [1] > > > > > > https://learn.microsoft.com/zh-tw/azure/databricks/sql/language-manual/sql-ref-syntax-aux-describe-catalog > > > > > > Best, > > > Yubin > > > > > > On Wed, Mar 20, 2024 at 2:15 PM Benchao Li > wrote: > > > > > > > > +1 (binding) > > > > > > > > gongzhongqiang 于2024年3月20日周三 11:40写道: > > > > > > > > > > +1 (non-binding) > > > > > > > > > > Best, > > > > > Zhongqiang Gong > > > > > > > > > > Yubin Li 于2024年3月19日周二 18:03写道: > > > > > > > > > > > Hi everyone, > > > > > > > > > > > > Thanks for all the feedback, I'd like to start a vote on the > > > FLIP-436: > > > > > > Introduce Catalog-related Syntax [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] > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-436%3A+Introduce+Catalog-related+Syntax > > > > > > [2] > > https://lists.apache.org/thread/10k1bjb4sngyjwhmfqfky28lyoo7sv0z > > > > > > > > > > > > Best regards, > > > > > > Yubin > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > Best, > > > > Benchao Li > > > > > >
Re: [VOTE] FLIP-445: Support dynamic parallelism inference for HiveSource
+1 (binding) Thanks, Zhu Ron Liu 于2024年4月26日周五 13:11写道: > +1(binding) > > Best, > Ron > > Rui Fan <1996fan...@gmail.com> 于2024年4月26日周五 12:55写道: > > > +1(binding) > > > > Best, > > Rui > > > > On Fri, Apr 26, 2024 at 10:26 AM Muhammet Orazov > > wrote: > > > > > Hey Xia, > > > > > > +1 (non-binding) > > > > > > Thanks and best, > > > Muhammet > > > > > > On 2024-04-26 02:21, Xia Sun wrote: > > > > Hi everyone, > > > > > > > > I'd like to start a vote on FLIP-445: Support dynamic parallelism > > > > inference > > > > for HiveSource[1] which has been discussed in this thread [2]. > > > > > > > > The vote will be open for at least 72 hours unless there is an > > > > objection or > > > > not enough votes. > > > > > > > > > > > > [1] > > > > > > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-445%3A+Support+dynamic+parallelism+inference+for+HiveSource > > > > [2] https://lists.apache.org/thread/4k1qx6lodhbkknkhjyl0lq9bx8fcpjvn > > > > > > > > > > > > Best, > > > > Xia > > > > > >