Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Bruno Cadonna
Hi Guozhang, Good point! I would propose to filter out DEAD stream threads in localThreadsMetadata() to get consistent results that do not depend on timing. I will update the KIP accordingly. Best, Bruno On 16.09.20 06:02, Guozhang Wang wrote: Thanks Bruno, your replies make sense to me. As

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Bruno Cadonna
Hi again, I just realized that if we filter out DEAD stream threads in localThreadsMetadata(), users cannot log the metadata of dying stream threads in the uncaught exception handler. I realized this thanks to the example Guozhang requested in the KIP. Thank you for that, Guozhang! Hence,

[jira] [Created] (KAFKA-10489) Committed consumer offsets not sent to consumer on rebalance

2020-09-16 Thread AD (Jira)
AD created KAFKA-10489: -- Summary: Committed consumer offsets not sent to consumer on rebalance Key: KAFKA-10489 URL: https://issues.apache.org/jira/browse/KAFKA-10489 Project: Kafka Issue Type: Bug

Re: [DISCUSSION] Upgrade system tests to python 3

2020-09-16 Thread Nikolay Izhikov
Hello, Guozhang. > I can help run the test suite once your PR is cleanly rebased to verify the > whole suite works Thank you for joining to the review. 1. PR rebased on the current trunk. 2. I triggered all tests in my private environment to verify them after rebase. Will inform you on

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Bruno Cadonna
Hi Walker, Thank you for the KIP! I like the motivation of the KIP and the method to request a shutdown of all Kafka Streams clients of Kafka Streams application. I think we really need such functionality to react on errors. However, I am not convinced that throwing an exception to shutdown a

Re: [DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-09-16 Thread Dongjin Lee
Hi John, > Although it's not great to have "special snowflakes" in the API, Choice B does seem safer in the short term. We would basically be proposing a temporary API to make the suppressed view queriable without a Materialized argument. Then, it seems like you prefer `KTable#suppress(Suppressed

Re: [DISCUSS] KIP-653: Upgrade log4j to log4j2

2020-09-16 Thread Dongjin Lee
Is there anyone who has suggestions or comments on this feature? Thanks, Dongjin On Wed, Aug 5, 2020 at 11:37 PM Dongjin Lee wrote: > Hi, Kafka dev, > > I hope to initiate the discussion of KIP-653, upgrading log4j to log4j2. > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-653%3A+Up

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-09-16 Thread Unmesh Joshi
Thanks Colin, the changes look good to me. One small thing. registration.lease.timeout.ms is the configuration on the controller side. It will be good to comment on how brokers know about it, to be able to send LeaseDurationMs in the heartbeat request, or else it can be added in the heartbeat respo

Re: [DISCUSS] KIP-567: Kafka Cluster Audit (new discussion)

2020-09-16 Thread Viktor Somogyi-Vass
Hi Mickael, Thanks for reviewing the KIP. 1.) I just wanted to follow the conventions used with the Authorizer as it is built in a similar fashion, although it's true that in KafkaServer we call the configure() method and the start() in the next line. This would be the same in Auditor and even si

Re: [DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-09-16 Thread John Roesler
Hi Dongjin, Yes, that's where I was leaning. Although, I'd prefer adding the option to Suppressed instead of adding a new argument to the method call. What do you think about: class Suppressed { + public Suppressed enableQuery(); } Since Suppressed already has `withName(String)`, it seems like

Re: [DISCUSS] KIP-653: Upgrade log4j to log4j2

2020-09-16 Thread Ismael Juma
Thanks for the KIP, Dongjin. Have we considered switching to the log4j2 logging config format by default and providing a mechanism to use the old format? It is likely that we will release 3.0 as the release after 2.7, so it would provide a good opportunity to move on from the legacy config format.

Re: [DISCUSS] KIP-567: Kafka Cluster Audit (new discussion)

2020-09-16 Thread Viktor Somogyi-Vass
One more after-thought on your second point (AbstractRequest): the reason I introduced it in the first place was that this way implementers can access request data. A use case can be if they want to audit a change in configuration or client quotas but not just acknowledge the fact that such an even

[jira] [Resolved] (KAFKA-10487) Fix edge case in Raft truncation protocol

2020-09-16 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10487?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-10487. - Resolution: Fixed > Fix edge case in Raft truncation protocol >

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Walker Carlson
Hello Guozhang and Bruno, Thanks for the feedback. I will respond in two parts but I would like to clarify that I am not tied down to any of these names, but since we are still deciding if we want to have an exception or not I would rather not get tripped up on choosing a name just yet. Guozhang

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #68

2020-09-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10487; Fetch response should return diverging epoch and end offset (#9290) -- Started by an SCM change Running as SYSTEM [EnvInje

Re: [DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-09-16 Thread Dongjin Lee
Hi John, It seems like the available alternatives in this point is clear: 1. Pass queriable name as a separate parameter (i.e., `KTable#suppress(Suppressed, String)`) 2. Make use of the Suppression processor name as a queryable name by adding `enableQuery` optional flag to `Suppressed`. However,

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Sophie Blee-Goldman
> > We guarantee that the metadata of the dead stream threads will be > returned by KafkaStreams#localThreadsMetadata() at least until the next > call to KafkaStreams#addStreamThread() or > KafkaStreams#removeStreamThread() after the stream thread transited to > DEAD This seems kind of tricky...

Re: [DISCUSS] KIP-653: Upgrade log4j to log4j2

2020-09-16 Thread Dongjin Lee
Hi Ismael, > Have we considered switching to the log4j2 logging config format by default and providing a mechanism to use the old format? As of present, the proposal leaves the default config format switching to sometime in the future. However, I think it is not a difficult task and is up to the

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Walker Carlson
Hello all again, I have updated the kip to no longer use an exception and instead add a method to the KafkaStreams class, this seems to satisfy everyone's concerns about how and when the functionality will be invoked. There is still a question over the name. We must decide between "shutdownApplic

[jira] [Created] (KAFKA-10490) Make constructors public for Admin API value objects

2020-09-16 Thread Noa Resare (Jira)
Noa Resare created KAFKA-10490: -- Summary: Make constructors public for Admin API value objects Key: KAFKA-10490 URL: https://issues.apache.org/jira/browse/KAFKA-10490 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Guozhang Wang
Hello Walker, Thanks for the updated KIP. Previously I'm also a bit hesitant on the newly added public exception to communicate user-requested whole app shutdown, but the reason I did not bring this up is that I feel there's still a need from operational aspects that we can differentiate the scena

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #67

2020-09-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10487; Fetch response should return diverging epoch and end offset (#9290) -- [...truncated 3.29 MB...] org.apache.kafka.streams

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Walker Carlson
Hello Guozhang, As for the logging I plan on having three logs. First, the client log that it is requesting an application shutdown, second, the leader log processId of the invoker, third, then the StreamRebalanceListener it logs that it is closing because of an `stream.appShutdown`. Hopefully thi

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-16 Thread Dhruvil Shah
Hi Satish, Harsha, Thanks for the KIP. Few questions below: 1. Could you describe how retention would work with this KIP and which threads are responsible for driving this work? I believe there are 3 kinds of retention processes we are looking at: (a) Regular retention for data in tiered storag

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #67

2020-09-16 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10487; Fetch response should return diverging epoch and end offset (#9290) -- [...truncated 6.53 MB...] org.apache.kafka.streams.

[jira] [Created] (KAFKA-10491) Check authorizations before other criteria in KafkaApis

2020-09-16 Thread David Arthur (Jira)
David Arthur created KAFKA-10491: Summary: Check authorizations before other criteria in KafkaApis Key: KAFKA-10491 URL: https://issues.apache.org/jira/browse/KAFKA-10491 Project: Kafka Issue

[jira] [Resolved] (KAFKA-10476) ignore it

2020-09-16 Thread AbdulRahman Mahmoud (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] AbdulRahman Mahmoud resolved KAFKA-10476. - Resolution: Invalid > ignore it > - > > Key: KAFKA-10476

[jira] [Created] (KAFKA-10492) Core Raft implementation

2020-09-16 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-10492: --- Summary: Core Raft implementation Key: KAFKA-10492 URL: https://issues.apache.org/jira/browse/KAFKA-10492 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-508: Make Suppression State Queriable - rebooted.

2020-09-16 Thread John Roesler
Hi Dongjin, Thanks for the reply. Yes, that’s correct, we added that method to name the operation. But the operation seems synonymous with the view produced the operation, right? During KIP-307, I remember thinking that it’s unfortunate the we had to have two different “name” concepts for the

[jira] [Created] (KAFKA-10493) Ktable out-of-order updates are not being ignored

2020-09-16 Thread Pedro Gontijo (Jira)
Pedro Gontijo created KAFKA-10493: - Summary: Ktable out-of-order updates are not being ignored Key: KAFKA-10493 URL: https://issues.apache.org/jira/browse/KAFKA-10493 Project: Kafka Issue Typ