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

2022-01-16 Thread Wenhao Ji
/manual/jmxConfig.html Thanks, Wenhao On Fri, Jan 14, 2022 at 12:12 AM Konstantin Knauf wrote: > > 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 Fli

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, > >

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

2022-01-11 Thread Wenhao Ji
> > > environment/filesystem? I only quickly googled the topic and [1,2] > > > suggest > > > > that this might be possible? > > > > > > > > [1] https://stackoverflow.com/a/16216956/6422562? > > > > [2] https://logback.qos.ch/manual/con

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

2022-01-10 Thread Wenhao Ji
Hi everyone, Hope you enjoyed the Holiday Season. I would like to start the discussion on the improvement purpose FLIP-210 [1] which aims to provide a way to change log levels at runtime to simplify issues and bugs detection as reported in the ticket FLINK-16478 [2]. Firstly, thanks Xingxing Di a

[jira] [Created] (FLINK-23899) Translate the "Elastic Scaling" page into chinese

2021-08-20 Thread Wenhao Ji (Jira)
Wenhao Ji created FLINK-23899: - Summary: Translate the "Elastic Scaling" page into chinese Key: FLINK-23899 URL: https://issues.apache.org/jira/browse/FLINK-23899 Project: Flink

Re: [DISCUSS] Support registering custom JobStatusListeners when scheduling a job

2021-08-17 Thread Wenhao Ji
ink/commit/2cab8bb1119162213632db984d2eb7529b8140e7#diff-4815dccc0cbf42d48f1668d4a076d19ec96c196bc562c3d2edae4d7bf9b9bd89R72 Thanks, Wenhao On Sat, Apr 10, 2021 at 9:32 PM Wenhao Ji wrote: > > Hi Till, > > Thanks for taking time out of your busy schedule. > I have created a POC for this feature. > > The c

Re: [DISCUSS] FLIP-172: Support custom transactional.id prefix in FlinkKafkaProducer

2021-07-09 Thread Wenhao Ji
6:32 PM Stephan Ewen wrote: > > Sounds good from my side, please go ahead. > > On Fri, Jun 25, 2021 at 5:31 PM Wenhao Ji wrote: >> >> Thanks Stephan and Piotr for your replies. It seems that there is no >> problem or concern about this feature. If there is no furthe

[RESULT] [VOTE] FLIP-172: Support custom transactional.id prefix in FlinkKafkaProducer

2021-07-09 Thread Wenhao Ji
Hi everyone, I am happy to announce that FLIP-172 [1] is approved. The vote [2] is now closed. There were five +1 votes, three of them were binding: - Dawid Wysakowicz (binding) - Piotr Nowojski (binding) - Arvid Heise (binding) - Yuan Mei (non-binding) - Daniel Lorych (non-binding) There was no

[VOTE] FLIP-172: Support custom transactional.id prefix in FlinkKafkaProducer

2021-06-28 Thread Wenhao Ji
Hi everyone, I would like to start a vote on FLIP-172 [1] which was discussed in this thread [2]. The vote will be open for at least 72 hours until July 1 unless there is an objection or not enough votes. Thanks, Wenhao [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-172%3A+Support+c

Re: [DISCUSS] FLIP-172: Support custom transactional.id prefix in FlinkKafkaProducer

2021-06-25 Thread Wenhao Ji
quick opinion from someone who worked specifically > > with Kafka, maybe Becket or Piotr? > > > > Best, > > Stephan > > > > > > On Sat, Jun 12, 2021 at 9:50 AM Wenhao Ji wrote: > > > >> Hi everyone, > >> > >> I would like to op

Re: [DISCUSS] Support specifying custom transactional.id prefix in FlinkKafkaProducer

2021-06-12 Thread Wenhao Ji
ers, > Till > > On Sat, Jun 5, 2021 at 9:48 AM Wenhao Ji wrote: > > > Hi everyone, > > > > Currently, the "transactional.id"s of the Kafka producers in > > FlinkKafkaProducer are generated based on the task name. This mechanism > has > > some

[DISCUSS] FLIP-172: Support custom transactional.id prefix in FlinkKafkaProducer

2021-06-12 Thread Wenhao Ji
Hi everyone, I would like to open this discussion thread to take about the FLIP-172 , which aims to provide a way to support specifying a custom transactional.id in the Flink

[DISCUSS] Support specifying custom transactional.id prefix in FlinkKafkaProducer

2021-06-05 Thread Wenhao Ji
Hi everyone, Currently, the "transactional.id"s of the Kafka producers in FlinkKafkaProducer are generated based on the task name. This mechanism has some limitations: - It will exceed Kafka's limitation if the task name is too long. (resolved in FLINK-17691) - They will very likely clash with

Re: [DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-05-04 Thread Wenhao Ji
requests you can do it via > reducing the `buffer.memory` option passed to KafkaProducer (via > FlinkKafkaProducer's "Properties producerConfig"). > > Piotrek > > [1] > > https://kafka.apache.org/24/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html > > pt.

[jira] [Created] (FLINK-22452) Support specifying custom transactional.id prefix in FlinkKafkaProducer

2021-04-25 Thread Wenhao Ji (Jira)
Wenhao Ji created FLINK-22452: - Summary: Support specifying custom transactional.id prefix in FlinkKafkaProducer Key: FLINK-22452 URL: https://issues.apache.org/jira/browse/FLINK-22452 Project: Flink

[DISCUSS] Add async backpressure support to FlinkKafkaProducer

2021-04-23 Thread Wenhao Ji
Hi everyone, I recently came across the following exception when dealing with a job failure, which uses the Flink as its sink. ``` org.apache.flink.streaming.connectors.kafka.FlinkKafkaException: Failed to send data to Kafka: Expiring #N record(s) for TOPIC-PARTITION:#N ms has passed since batch

Re: [DISCUSS] Support registering custom JobStatusListeners when scheduling a job

2021-04-10 Thread Wenhao Ji
e upcoming feature freeze (just > for your information). > > Cheers, > Till > > On Wed, Mar 10, 2021 at 3:40 PM Wenhao Ji wrote: > > > Hi Till. Indeed, there is no proper solution now other than the polling > > method. It is painful to have such code in our platform

Re: [DISCUSS] Support registering custom JobStatusListeners when scheduling a job

2021-03-10 Thread Wenhao Ji
Hi Till. Indeed, there is no proper solution now other than the polling method. It is painful to have such code in our platform since it consumes a lot of resources to keep the polling run periodically when there are hundreds of Flink clusters to maintain. A lot of pollings are actually useless as