Re: Failed to commit consumer offsets for checkpoint 108768

2023-03-17 Thread Sucheth S
Hi, Got it. Thanks for the update. We are using 1.15.0. Are there any measures we need to take to upgrade from 1.15 to 1.16 ? On Fri, Mar 17, 2023 at 8:06 PM Ken Krugler wrote: > Hi Suchet, > > Are you running with Flink 1.15? There was a bug in Kafka < > https://issues.apache.org/jira/browse/

Re: Failed to commit consumer offsets for checkpoint 108768

2023-03-17 Thread Ken Krugler
Hi Suchet, Are you running with Flink 1.15? There was a bug in Kafka , which showed up in Flink 1.15 due to an upgrade in Kafka dependencies. It’s tracked by this Flink Jira issue , and has be

Failed to commit consumer offsets for checkpoint 108768

2023-03-17 Thread Sucheth S
Hi, All of our Flink jobs started throwing the below error recently, Can someone please help what could be the reason for this? Anyone experienced below issue before ? org.apache.kafka.clients.consumer.RetriableCommitFailedException: Failed to commit consumer offsets for checkpoint 108757 org.ap

[jira] [Created] (FLINK-31510) Use getMemorySize instead of getMemory

2023-03-17 Thread Shilun Fan (Jira)
Shilun Fan created FLINK-31510: -- Summary: Use getMemorySize instead of getMemory Key: FLINK-31510 URL: https://issues.apache.org/jira/browse/FLINK-31510 Project: Flink Issue Type: Improvement

[DISCUSS] FLIP-304: Pluggable failure handling for Apache Flink

2023-03-17 Thread Shammon FY
Hi Panagiotis Thank you for starting this discussion. I think this FLIP is valuable and can help user to analyze the causes of job failover better! I have two comments as follows 1. How about adding more job information in FailureListenerContext? For example, job vertext, subtask, taskmanager lo

[jira] [Created] (FLINK-31509) REST Service missing sessionAffinity causes job run failure with HA cluster

2023-03-17 Thread Emmanuel Leroy (Jira)
Emmanuel Leroy created FLINK-31509: -- Summary: REST Service missing sessionAffinity causes job run failure with HA cluster Key: FLINK-31509 URL: https://issues.apache.org/jira/browse/FLINK-31509 Proje

[DISCUSS] FLIP-304: Pluggable failure handling for Apache Flink

2023-03-17 Thread Panagiotis Garefalakis
Hi everyone, This FLIP [1] proposes a pluggable interface for failure handling allowing users to implement custom failure logic using the plugin framework. Motivated by existing proposals [2] and tickets [3], this enables use-cases like: assigning particular types to failures (e.g., User or System

Is there a way to control the parallelism of auto-generated Flink operators of the FlinkSQL job graph?

2023-03-17 Thread Elkhan Dadashov
Dear Flink developers, Wanted to check, if there is a way to control the parallelism of auto-generated Flink operators of the FlinkSQL job graph? In Java API, it is possible to have full control of the parallelism of each operator. On FlinkSQL some source and sink connectors support `source.para

[jira] [Created] (FLINK-31508) FLIP-304: Pluggable failure handling for Apache Flink

2023-03-17 Thread Panagiotis Garefalakis (Jira)
Panagiotis Garefalakis created FLINK-31508: -- Summary: FLIP-304: Pluggable failure handling for Apache Flink Key: FLINK-31508 URL: https://issues.apache.org/jira/browse/FLINK-31508 Project: Fli

Re: [DISCUSS] FLIP-299 Pub/Sub Lite Connector

2023-03-17 Thread Daniel Collins
> would the repository ... be removed ... ? Yes, I would remove it once it is merged into a version of flink that is supported by GCP dataproc. It exists now (and I am creating releases and maven artifacts for it) to unblock users in the interim period. -Daniel On Thu, Mar 16, 2023 at 3:32 PM Ma

[jira] [Created] (FLINK-31507) Move execution logic of ShowOperation out from TableEnvironmentImpl

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31507: --- Summary: Move execution logic of ShowOperation out from TableEnvironmentImpl Key: FLINK-31507 URL: https://issues.apache.org/jira/browse/FLINK-31507 Project: Flink Is

[jira] [Created] (FLINK-31505) Move execution logic of DropOperation out from TableEnvironmentImpl

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31505: --- Summary: Move execution logic of DropOperation out from TableEnvironmentImpl Key: FLINK-31505 URL: https://issues.apache.org/jira/browse/FLINK-31505 Project: Flink Is

[jira] [Created] (FLINK-31506) Move execution logic of AlterOperation out from TableEnvironmentImpl

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31506: --- Summary: Move execution logic of AlterOperation out from TableEnvironmentImpl Key: FLINK-31506 URL: https://issues.apache.org/jira/browse/FLINK-31506 Project: Flink I

[jira] [Created] (FLINK-31504) Move execution logic of CreateOperation out from TableEnvironmentImpl

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31504: --- Summary: Move execution logic of CreateOperation out from TableEnvironmentImpl Key: FLINK-31504 URL: https://issues.apache.org/jira/browse/FLINK-31504 Project: Flink

[jira] [Created] (FLINK-31503) "org.apache.beam.sdk.options.PipelineOptionsRegistrar: Provider org.apache.beam.sdk.options.DefaultPipelineOptionsRegistrar not a subtype" is thrown when executing Pytho

2023-03-17 Thread Dian Fu (Jira)
Dian Fu created FLINK-31503: --- Summary: "org.apache.beam.sdk.options.PipelineOptionsRegistrar: Provider org.apache.beam.sdk.options.DefaultPipelineOptionsRegistrar not a subtype" is thrown when executing Python UDFs in SQL Client Key:

[VOTE] Release 1.17.0, release candidate #3

2023-03-17 Thread Qingsheng Ren
Hi everyone, Please review and vote on the release candidate #3 for the version 1.17.0, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which includes: * JIRA release notes [1],

[jira] [Created] (FLINK-31502) Limit the number of concurrent scale operations to reduce cluster churn

2023-03-17 Thread Maximilian Michels (Jira)
Maximilian Michels created FLINK-31502: -- Summary: Limit the number of concurrent scale operations to reduce cluster churn Key: FLINK-31502 URL: https://issues.apache.org/jira/browse/FLINK-31502 P

[jira] [Created] (FLINK-31501) Move SqlCreateView conversion logic to SqlCreateViewConverter

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31501: --- Summary: Move SqlCreateView conversion logic to SqlCreateViewConverter Key: FLINK-31501 URL: https://issues.apache.org/jira/browse/FLINK-31501 Project: Flink Issue Ty

[jira] [Created] (FLINK-31500) Move SqlAlterTableSchema conversion logic to AlterTableSchemaConverter

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31500: --- Summary: Move SqlAlterTableSchema conversion logic to AlterTableSchemaConverter Key: FLINK-31500 URL: https://issues.apache.org/jira/browse/FLINK-31500 Project: Flink

[jira] [Created] (FLINK-31499) Move SqlCreateTable conversion logic to SqlCreateTableConverter

2023-03-17 Thread Jark Wu (Jira)
Jark Wu created FLINK-31499: --- Summary: Move SqlCreateTable conversion logic to SqlCreateTableConverter Key: FLINK-31499 URL: https://issues.apache.org/jira/browse/FLINK-31499 Project: Flink Issue

[DISCUSS] FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source

2023-03-17 Thread Hongshun Wang
Hi everyone, I would like to start a discussion on FLIP-288:Enable Dynamic Partition Discovery by Default in Kafka Source[1]. As described in mail thread[2], dynamic partition discovery is disabled by default and users have to explicitly specify the interval of discovery in order to turn it on. B

[jira] [Created] (FLINK-31498) DeclartiveSlotManager always request redundant task manager when resource is not enough

2023-03-17 Thread Weihua Hu (Jira)
Weihua Hu created FLINK-31498: - Summary: DeclartiveSlotManager always request redundant task manager when resource is not enough Key: FLINK-31498 URL: https://issues.apache.org/jira/browse/FLINK-31498 Pro

[jira] [Created] (FLINK-31497) Drop the deprecated CatalogViewImpl

2023-03-17 Thread Aitozi (Jira)
Aitozi created FLINK-31497: -- Summary: Drop the deprecated CatalogViewImpl Key: FLINK-31497 URL: https://issues.apache.org/jira/browse/FLINK-31497 Project: Flink Issue Type: Technical Debt

[jira] [Created] (FLINK-31496) FLIP-293: Introduce Flink Jdbc Driver For Sql Gateway

2023-03-17 Thread Fang Yong (Jira)
Fang Yong created FLINK-31496: - Summary: FLIP-293: Introduce Flink Jdbc Driver For Sql Gateway Key: FLINK-31496 URL: https://issues.apache.org/jira/browse/FLINK-31496 Project: Flink Issue Type: I

[jira] [Created] (FLINK-31495) Improve metrics tab on flink ui

2023-03-17 Thread zhang haoyan (Jira)
zhang haoyan created FLINK-31495: Summary: Improve metrics tab on flink ui Key: FLINK-31495 URL: https://issues.apache.org/jira/browse/FLINK-31495 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-31494) Introduce OperationExecutor for SqlToOperationConverter

2023-03-17 Thread xzw0223 (Jira)
xzw0223 created FLINK-31494: --- Summary: Introduce OperationExecutor for SqlToOperationConverter Key: FLINK-31494 URL: https://issues.apache.org/jira/browse/FLINK-31494 Project: Flink Issue Type: Sub

[RESULT][VOTE] FLIP-293: Introduce Flink Jdbc Driver For Sql Gateway

2023-03-17 Thread Shammon FY
Hi everyone: I am happy to announce that FLIP-293: Introduce Flink Jdbc Driver For Sql Gateway [1] has been accepted. There are 6 binding votes and 5 non-binding votes: - Benchao Li (binding) - Jingsong Li (binding) - Jing Ge (binding) - weijie guo (binding) - Leonard Xu (binding) - Dong Lin (bin