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

2020-08-28 Thread Colin McCabe
On Fri, Aug 28, 2020, at 19:36, Unmesh Joshi wrote: > Hi Colin, > > There were a few of questions I had.. Hi Unmesh, Thanks for the response. > > 1. Were my comments on the broker lease implementation (and corresponding > prototype) appropriate and do we need to change the KIP > description

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

2020-08-28 Thread Unmesh Joshi
Hi Colin, There were a few of questions I had.. 1. Were my comments on the broker lease implementation (and corresponding prototype) appropriate and do we need to change the KIP description accordingly?. 2. How will broker epochs be generated? I am assuming it can be the committed log offset

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

2020-08-28 Thread Colin McCabe
Hi all, I'm thinking of calling a vote on KIP-631 on Monday. Let me know if there's any more comments I should address before I start the vote. cheers, Colin On Tue, Aug 11, 2020, at 05:39, Unmesh Joshi wrote: > >>Hi Unmesh, > >>Thanks, I'll take a look. > Thanks. I will be adding more to the

Re: [DISCUSS] KIP-654 Aborting Transaction with non-flushed data should throw a non-fatal Exception

2020-08-28 Thread Jason Gustafson
Hi Gokul, Thanks, I think it makes sense to use a separate exception type. +1 on Sophie's suggestion for `TransactionAbortedException`. Extending from `RetriableException` seems reasonable as well. I guess the only question is whether it's safe to catch it as a `RetriableException` and apply

[DISCUSS] KIP-666: Add Instant-based methods to ReadOnlySessionStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to discuss the following proposal to align IQ Session Store API with the Window Store one. https://cwiki.apache.org/confluence/display/KAFKA/KIP-666%3A+Add+Instant-based+methods+to+ReadOnlySessionStore Looking forward to your feedback. Cheers, Jorge.

[DISCUSS] KIP-667: Remove deprecated methods from ReadOnlyWindowStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'd like to propose these changes to the Window Store API. https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore As these changes involve removing deprecated methods, this KIP is targeting the next major release v3.0.

Re: [VOTE] KIP-554: Add Broker-side SCRAM Config API

2020-08-28 Thread Colin McCabe
Hi Ron, Thanks for the update. I agree with all of these changes, except I think we should discuss this one further: On Wed, Aug 26, 2020, at 14:59, Ron Dagostino wrote: > > 2. We added a restriction to not allow users who authenticated using > delegation tokens to create or update user SCRAM

confluent-kafka python library is not working with ubutu14 and python3

2020-08-28 Thread Kafka Shil
I am using "confluent-kafka==1.0.1". It works fine when I am using py3 and ubuntu18, but fails with py3 and ubuntu14. I get the following error. Traceback (most recent call last): File "/usr/local/lib/python3.4/dist-packages/metrics_agent/kafka_writer.py", line 147, in enqueue_for_topic

[jira] [Created] (KAFKA-10445) Align IQ SessionStore API with Instant-based methods as ReadOnlyWindowStore

2020-08-28 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-10445: Summary: Align IQ SessionStore API with Instant-based methods as ReadOnlyWindowStore Key: KAFKA-10445 URL:

Re: [DISCUSS] KIP-640 Add log compression analysis tool

2020-08-28 Thread Christopher Beard
Hi Alex, thanks for the question! In the simplest sense, the tool doesn't know anything about the messages in the log or any particular batch. The tool would compress the encrypted data to measure the resulting size, but the results would likely show no reduction in data size. Effectively, the

[jira] [Created] (KAFKA-10444) Migrate PR jobs to new Apache Infra Jenkins

2020-08-28 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-10444: --- Summary: Migrate PR jobs to new Apache Infra Jenkins Key: KAFKA-10444 URL: https://issues.apache.org/jira/browse/KAFKA-10444 Project: Kafka Issue Type: Task

Re: [DISCUSS] KIP-406: GlobalStreamThread should honor custom reset policy

2020-08-28 Thread Navinder Brar
Gentle ping. ~ Navinder On Wednesday, 19 August, 2020, 06:59:58 pm IST, Navinder Brar wrote: Thanks Matthias & John,  I am glad we are converging towards an understanding. So, to summarize,  we will still keep treating this change in KIP and instead of providing a reset

Re: KIP idea: Separate publish request from the subscribe request

2020-08-28 Thread Ming Liu
Hi Guozhang, Yes, the goal is to reduce latency of produce latency from the consumer fetch requests. So the best separate in theory would be: 1. Socket server 1: All produce requests and all follower fetch requests (ACK=ALL) 2. Socket server 2: All other requests (metadata,

Re: [DISCUSS] KIP-664: Provide tooling to detect and abort hanging transactions

2020-08-28 Thread Robert Barrett
Hi Jason, Thanks for this KIP, I think this will be a huge operational improvement and overall it looks great to me. I'm not sure how much value the MaxActiveTransactionDuration metric adds, given that we have the --find-hanging option in the tool. As you mention, instances of these transactions

Re: Grant me a permission to create KIP

2020-08-28 Thread Jun Rao
Hi, Rupesh, Thanks for your interest. Just gave you the wiki permission. Jun On Fri, Aug 28, 2020 at 8:09 AM rupesh patel wrote: > Hi Team, > > Please provide me permission to create a KIP > > My Wiki id: rupeshkumarpatel02 > > > Thanks, > Rupesh Kumar Patel >

Grant me a permission to create KIP

2020-08-28 Thread rupesh patel
Hi Team, Please provide me permission to create a KIP My Wiki id: rupeshkumarpatel02 Thanks, Rupesh Kumar Patel

Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor

2020-08-28 Thread Ryanne Dolan
Thanks Mickael, the KIP makes sense to me, esp for cases where an external system (like cruise control or an operator) knows more about the target cluster state than the broker does. Ryanne On Thu, Aug 20, 2020, 10:46 AM Mickael Maison wrote: > Hi, > > I've created KIP-660 to make the replica

Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor

2020-08-28 Thread Robert Barrett
Hi Mickael, Thanks for the KIP! One question I have is around failure cases. Are ReplicaAssignor implementations expected to always compute an assignment, or is it possible for them to have unsatisfiable conditions? One example I can think of is a requirement that at least one partition be

[jira] [Resolved] (KAFKA-10307) Topology cycles in KTableKTableForeignKeyInnerJoinMultiIntegrationTest#shouldInnerJoinMultiPartitionQueryable

2020-08-28 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-10307. -- Resolution: Not A Bug This bug report was based on an invalid assumption that topologies

Re: [DISCUSS] KIP-567: Kafka Cluster Audit

2020-08-28 Thread Viktor Somogyi-Vass
Hi folks, I have a use-case and a non-trivial implementation with Apache Atlas for this KIP and since this kip seems to be dormant for a while now, I'd take it over and drive it to completion if you don't mind. The current state of the PoC can be found on my fork at

Re: [DISCUSSION] Upgrade system tests to python 3

2020-08-28 Thread Nikolay Izhikov
Hello! Any feedback on this? What I should additionally do to prepare system tests migration? > 24 авг. 2020 г., в 11:17, Nikolay Izhikov написал(а): > > Hello. > > PR [1] is ready. > Please, review. > > But, I need help with the two following questions: > > 1. We need a new release of

Backports of KAFKA-9839 onto 2.2, 2.3 and 2.4

2020-08-28 Thread Viktor Somogyi-Vass
Hi folks, I have backported KAFKA-9839 onto 2.2, 2.3 and 2.4 respectively as we detected it with a customer running 2.2. Would anyone please help me review it? The PRs are: https://github.com/apache/kafka/pull/9150

Re: [DISCUSS] KIP-647: Add ability to handle late messages in streams-aggregation

2020-08-28 Thread Matthias J. Sax
In general, I like Bruno's proposal. Was also checking how Flink is handling this case, and similar to Bruno's proposal, they have a so-called "side output": https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/operators/windows.html#getting-late-data-as-a-side-output The only