Re: Flink native k8s integration vs. operator

2022-01-13 Thread Xintong Song
Thanks for volunteering to drive this effort, Marton, Thomas and Gyula. Looking forward to the public discussion. Please feel free to reach out if there's anything you need from us. Thank you~ Xintong Song On Fri, Jan 14, 2022 at 8:27 AM Chenya Zhang wrote: > Thanks Thomas, Gyula, and

[jira] [Created] (FLINK-25654) Remove the redundant lock in SortMergeResultPartition

2022-01-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25654: --- Summary: Remove the redundant lock in SortMergeResultPartition Key: FLINK-25654 URL: https://issues.apache.org/jira/browse/FLINK-25654 Project: Flink Issue

[jira] [Created] (FLINK-25653) Move buffer recycle in SortMergeSubpartitionReader out of lock to avoid deadlock

2022-01-13 Thread Yingjie Cao (Jira)
Yingjie Cao created FLINK-25653: --- Summary: Move buffer recycle in SortMergeSubpartitionReader out of lock to avoid deadlock Key: FLINK-25653 URL: https://issues.apache.org/jira/browse/FLINK-25653

Re: [DISCUSS] FLIP-203: Incremental savepoints

2022-01-13 Thread Yu Li
Thanks for the update, Piotr! > Is `state.backend.incremental` the only configuration parameter that can be > used in this context? According to FLIP-193 [1], all the existing checkpoint configurations are actually for *Snapshot*, ownership (lifecycle) is the only difference between Checkpoints

[jira] [Created] (FLINK-25652) Can "duration“ and ”received records" be updated at the same time in WebUI's task detail ?

2022-01-13 Thread jeff-zou (Jira)
jeff-zou created FLINK-25652: Summary: Can "duration“ and ”received records" be updated at the same time in WebUI's task detail ? Key: FLINK-25652 URL: https://issues.apache.org/jira/browse/FLINK-25652

Re: [DISCUSS] Creating an external connector repository

2022-01-13 Thread Martijn Visser
Hi everyone, If you have any more comments or questions, do let me know. Else I'll open up a vote thread next week. Best regards, Martijn On Tue, 11 Jan 2022 at 20:13, Martijn Visser wrote: > Good question: we want to use the same setup as we currently have for > Flink, so using the existing

Re: [DISCUSS] Moving connectors from Flink to external connector repositories

2022-01-13 Thread Martijn Visser
Hi everyone, If you have any more comments or questions, please let me know. Else I would open up a vote on this thread in the next couple of days. Best regards, Martijn On Thu, 6 Jan 2022 at 09:45, Qingsheng Ren wrote: > Thanks Martijn for driving this! > > I’m +1 for Martijn’s proposal.

Re: [DISCUSS] Releasing Flink 1.14.3

2022-01-13 Thread Martijn Visser
Hi all, We still need a couple of PMCs to validate the release and put out their vote. The vote thread can be found at https://lists.apache.org/thread/cqn8p7kv4tbc4hn4czjvzfcd905jztro Best regards, Martijn On Tue, 11 Jan 2022 at 16:29, Martijn Visser wrote: > Hi Thomas, > > Thanks! I'll

[jira] [Created] (FLINK-25651) Flink1.14.2 DataStream Connectors Kafka Deserializer example method uses the wrong parameter

2022-01-13 Thread shouzuo meng (Jira)
shouzuo meng created FLINK-25651: Summary: Flink1.14.2 DataStream Connectors Kafka Deserializer example method uses the wrong parameter Key: FLINK-25651 URL: https://issues.apache.org/jira/browse/FLINK-25651

Re: [DISCUSS] FLIP-211: Kerberos delegation token framework

2022-01-13 Thread Junfan Zhang
Hi G Thanks for your explain in detail. I have gotten your thoughts, and any way this proposal is a great improvement. Looking forward to your implementation and i will keep focus on it. Thanks again. Best JunFan. On Jan 13, 2022, 9:20 PM +0800, Gabor Somogyi , wrote: > Just to confirm

Re: Flink native k8s integration vs. operator

2022-01-13 Thread Chenya Zhang
Thanks Thomas, Gyula, and Marton for driving this effort! It would greatly ease the adoption of Apache Flink on Kubernetes and help to address the current operational pain points as mentioned. Look forward to the proposal and more discussions! Best, Chenya On Thu, Jan 13, 2022 at 12:15 PM Márton

Re: [DISCUSS] FLIP-208: Update KafkaSource to detect EOF based on de-serialized record

2022-01-13 Thread Dong Lin
Hi Qingsheng, Thank you for extending this FLIP to support this feature with Table/SQL. I also prefer the 2nd option over the 1st option. This is because the 1st option requires the user to additionally specify an identifier in addition to defining the RecordEvaluator themselves. Note that this

Re: Flink native k8s integration vs. operator

2022-01-13 Thread Márton Balassi
Hi All, I am pleased to see the level of enthusiasm and technical consideration already emerging in this thread. I wholeheartedly support building an operator and endorsing it via placing it under the Apache Flink umbrella (as a separate repository) as the current lack of it is clearly becoming

Re: [FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Maciej Obuchowski
Hello, I'm an OpenLineage committer - and previously, a minor Flink contributor. OpenLineage community is very interested in conversation about Flink metadata, and we'll be happy to cooperate with the Flink community. Best, Maciej Obuchowski czw., 13 sty 2022 o 18:12 Martijn Visser

Re: [FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Martijn Visser
Hi all, @Andrew thanks for sharing that! @Tero good point, I should have clarified the purpose. I want to understand what "metadata platforms" tools are used or evaluated by the Flink community, what's their purpose for using such a tool (is it as a generic catalogue, as a data discovery tool,

Re: [DISCUSS] FLIP-210: Change logging level dynamically at runtime

2022-01-13 Thread Konstantin Knauf
Thanks, Wenhao. On Thu, Jan 13, 2022 at 4:19 PM Wenhao Ji wrote: > It seems that we have reached a consensus that the proposal will not > be implemented in Flink. I will mark the FLIP as discarded if there > are no objections. > > Thanks, everyone, for joining the discussion again! > > Wenhao >

Re: [DISCUSS] Future of Per-Job Mode

2022-01-13 Thread Thomas Weise
Regarding session mode: ## Session Mode * main() method executed in client Session mode also supports execution of the main method on Jobmanager with submission through REST API. That's how Flinkk k8s operators like [1] work. It's actually an important capability because it allows for allocation

Re: [FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Andrew Otto
Hello! The Wikimedia Foundation is currently doing a similar evaluation (although we are not currently including any Flink considerations). https://wikitech.wikimedia.org/wiki/Data_Catalog_Application_Evaluation_Rubric More details will be published there as folks keep working on this. Hope

Re: [FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Pedro Silva
Hello, I'm part of the DataHub community and working in collaboration with the company behind it: http://acryldata.io Happy to have a conversation or clarify any questions you may have on DataHub :) Have a nice day! Em qui., 13 de jan. de 2022 às 15:33, Andrew Otto escreveu: > Hello! The

[FEEDBACK] Metadata Platforms / Catalogs / Lineage integration

2022-01-13 Thread Martijn Visser
Hi everyone, I'm currently checking out different metadata platforms, such as Amundsen [1] and Datahub [2]. In short, these types of tools try to address problems related to topics such as data discovery, data lineage and an overall data catalogue. I'm reaching out to the Dev and User mailing

Re: [DISCUSS] FLIP-210: Change logging level dynamically at runtime

2022-01-13 Thread Wenhao Ji
It seems that we have reached a consensus that the proposal will not be implemented in Flink. I will mark the FLIP as discarded if there are no objections. Thanks, everyone, for joining the discussion again! Wenhao On Tue, Jan 11, 2022 at 11:12 PM Wenhao Ji wrote: > > Hi all, > > Yes, indeed.

[jira] [Created] (FLINK-25650) Document unaligned checkpoints performance limitations (larger records/flat map/timers/...)

2022-01-13 Thread Anton Kalashnikov (Jira)
Anton Kalashnikov created FLINK-25650: - Summary: Document unaligned checkpoints performance limitations (larger records/flat map/timers/...) Key: FLINK-25650 URL:

[jira] [Created] (FLINK-25649) Scheduling jobs fails with org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException

2022-01-13 Thread Gil De Grove (Jira)
Gil De Grove created FLINK-25649: Summary: Scheduling jobs fails with org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException Key: FLINK-25649 URL:

Re: [DISCUSS] FLIP-211: Kerberos delegation token framework

2022-01-13 Thread Gabor Somogyi
Just to confirm keeping "security.kerberos.fetch.delegation-token" is added to the doc. BR, G On Thu, Jan 13, 2022 at 1:34 PM Gabor Somogyi wrote: > Hi JunFan, > > > By the way, maybe this should be added in the migration plan or > intergation section in the FLIP-211. > > Going to add this

[jira] [Created] (FLINK-25648) Redundant to querying deployment when creating task manager pod

2022-01-13 Thread Yuan Huang (Jira)
Yuan Huang created FLINK-25648: --- Summary: Redundant to querying deployment when creating task manager pod Key: FLINK-25648 URL: https://issues.apache.org/jira/browse/FLINK-25648 Project: Flink

[jira] [Created] (FLINK-25647) Improve JSON format handling and support

2022-01-13 Thread Martijn Visser (Jira)
Martijn Visser created FLINK-25647: -- Summary: Improve JSON format handling and support Key: FLINK-25647 URL: https://issues.apache.org/jira/browse/FLINK-25647 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-211: Kerberos delegation token framework

2022-01-13 Thread Gabor Somogyi
Hi JunFan, > By the way, maybe this should be added in the migration plan or intergation section in the FLIP-211. Going to add this soon. > Besides, I have a question that the KDC will collapse when the cluster reached 200 nodes you described in the google doc. Do you have any attachment or

[jira] [Created] (FLINK-25646) Document buffer debloating issues with high parallelism

2022-01-13 Thread Anton Kalashnikov (Jira)
Anton Kalashnikov created FLINK-25646: - Summary: Document buffer debloating issues with high parallelism Key: FLINK-25646 URL: https://issues.apache.org/jira/browse/FLINK-25646 Project: Flink

Re: [DISCUSS] FLIP-211: Kerberos delegation token framework

2022-01-13 Thread 张俊帆
Hi G Thanks for your quick reply. I think reserving the config of *security.kerberos.fetch.delegation-token* and simplifying disable the token fetching is a good idea.By the way, maybe this should be added in the migration plan or intergation section in the FLIP-211. Besides, I have a question

Re: [DISCUSS] FLIP-203: Incremental savepoints

2022-01-13 Thread Piotr Nowojski
Hi, Thanks for the comments and questions. Starting from the top: Seth: good point about schema evolution. Actually, I have a very similar question to State Processor API. Is it the same scenario in this case? Should it also be working with checkpoints but might be just untested? And next

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

2022-01-13 Thread Xingbo Huang
+1 (non-binding) - Verified checksums and signatures - Verified Python wheel package contents - Pip install apache-flink-libraries source package and apache-flink wheel package in Mac - Run the examples from Python Table API Tutorial[1] in Python REPL [1]

[jira] [Created] (FLINK-25645) UnsupportedOperationException would thrown out when hash shuffle by a field with array type

2022-01-13 Thread Jing Zhang (Jira)
Jing Zhang created FLINK-25645: -- Summary: UnsupportedOperationException would thrown out when hash shuffle by a field with array type Key: FLINK-25645 URL: https://issues.apache.org/jira/browse/FLINK-25645

[jira] [Created] (FLINK-25644) Introduce interfaces between file-table-store and flink connector sink

2022-01-13 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-25644: Summary: Introduce interfaces between file-table-store and flink connector sink Key: FLINK-25644 URL: https://issues.apache.org/jira/browse/FLINK-25644 Project:

[jira] [Created] (FLINK-25643) Introduce Predicate to table store

2022-01-13 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-25643: Summary: Introduce Predicate to table store Key: FLINK-25643 URL: https://issues.apache.org/jira/browse/FLINK-25643 Project: Flink Issue Type: Sub-task

Re: [DISCUSS] FLIP-211: Kerberos delegation token framework

2022-01-13 Thread Gabor Somogyi
Hi Junfan, Thanks for investing your time to make this feature better. I've had a look at FLINK-21700 and now I think I see your point (plz correct me if I misunderstood something). According to the actual plans *security.kerberos.fetch.delegation-token* is intended to be removed because

Re: [DISCUSS] FLIP-208: Update KafkaSource to detect EOF based on de-serialized record

2022-01-13 Thread Qingsheng Ren
Thanks Dong for the explanation! I agree with Dong’s idea of keeping the consistency of APIs for setting configurations, so I think it’s acceptable for me to pass the record evaluator from XXXSourceBuilder and embed it into SourceReaderBase. Also considering current usage of the

Re: [DISCUSS] Future of Per-Job Mode

2022-01-13 Thread Biao Geng
Hi Konstantin, Thanks a lot for starting this discussion! I hope my thoughts and experiences why users use Per-Job Mode, especially in YARN can help: #1. Per-job mode makes managing dependencies easier: I have met some customers who used Per-Job Mode to submit jobs with a lot of local

RE: OutOfMemoryError: Java heap space while implmentating flink sql api

2022-01-13 Thread Ronak Beejawat (rbeejawa)
HI Martijn, I posted the below query both the places(flink mailing list and stack overflow) to get a quick response on it. Please let me know the exact poc / mailing list to post my quries if it is causing trouble, so at least we can get quick acknowledgement on the issues reported. Ok let me

RE: Could not find any factory for identifier 'jdbc'

2022-01-13 Thread Ronak Beejawat (rbeejawa)
Hi Roman, Chesnay PFB screenshot for jdbc connector availability in bundle jar as I mentioned earlier it didn't worked even than, so I tried putting it in inside flink lib directory as mentioned in below article link then it resolved the issue. [cid:image001.png@01D80864.522974B0]

[DISCUSS] Merge flink-connector-testing into flink-connector-test-utils

2022-01-13 Thread Qingsheng Ren
Hi all, I’d like to start a discussion about merging two utilization modules "flink-connector-testing" and "flink-connector-test-utils". Both modules are designed for providing connector testing infrastructures and helper classes, so merging these two modules could simplify dependencies of

[DISCUSS] Future of Per-Job Mode

2022-01-13 Thread Konstantin Knauf
Hi everyone, I would like to discuss and understand if the benefits of having Per-Job Mode in Apache Flink outweigh its drawbacks. *# Background: Flink's Deployment Modes* Flink currently has three deployment modes. They differ in the following dimensions: * main() method executed on Jobmanager

Re: Flink native k8s integration vs. operator

2022-01-13 Thread Konstantin Knauf
Hi Thomas, Yes, I was referring to a separate repository under Apache Flink. Cheers, Konstantin On Thu, Jan 13, 2022 at 6:19 AM Thomas Weise wrote: > Hi everyone, > > Thanks for the feedback and discussion. A few additional thoughts: > > [Konstantin] > With respect to common lifecycle

[jira] [Created] (FLINK-25642) The standalone deployment mode should also support the parameter pipeline.jars

2022-01-13 Thread john (Jira)
john created FLINK-25642: Summary: The standalone deployment mode should also support the parameter pipeline.jars Key: FLINK-25642 URL: https://issues.apache.org/jira/browse/FLINK-25642 Project: Flink

[jira] [Created] (FLINK-25641) Unexpected aggregate plan after load hive module

2022-01-13 Thread Jing Zhang (Jira)
Jing Zhang created FLINK-25641: -- Summary: Unexpected aggregate plan after load hive module Key: FLINK-25641 URL: https://issues.apache.org/jira/browse/FLINK-25641 Project: Flink Issue Type: