[VOTE] KIP-726: Make the "cooperative-sticky, range" as the default assignor

2021-06-02 Thread Luke Chen
Hi all, I'd like to call for a vote on KIP-726: Make the "cooperative-sticky, range" as the default assignor. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177048248 This KIP is still waiting for the prerequisite stories to get completed, but it should be soon. Hopefully this

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #191

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 473124 lines...] [2021-06-03T02:43:50.721Z] [2021-06-03T02:43:50.721Z] SaslPlainSslEndToEndAuthorizationTest >

[jira] [Created] (KAFKA-12883) Adress KIP-100 type constraints now that Java 7 support is dropped

2021-06-02 Thread Jira
Xavier Léauté created KAFKA-12883: - Summary: Adress KIP-100 type constraints now that Java 7 support is dropped Key: KAFKA-12883 URL: https://issues.apache.org/jira/browse/KAFKA-12883 Project: Kafka

Re: [DISCUSS] KIP-726: Make the CooperativeStickyAssignor as the default assignor

2021-06-02 Thread Luke Chen
Hi Sophie, Thanks for the reminder. Yes, I was thinking this KIP doesn't have to be put into a major release since it will be fully backward compatible, so no need to push it. Currently, if we want to work on this KIP, we need KAFKA-12477 and KAFKA-12487. But you're right, we can at least try our

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #190

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 473386 lines...] [2021-06-03T00:32:44.462Z] [2021-06-03T00:32:44.462Z] DynamicConnectionQuotaTest > testDynamicConnectionQuota() STARTED

Re: [VOTING] KIP-693: Client-side Circuit Breaker for Partition Write Errors

2021-06-02 Thread lobo xu
+1. I think this feature is very helpful to the community. In the event of performance stress or failure on some brokers, traffic can be imported to other normal partitions. lobo > 在 2021年6月2日,10:18,Guoqiang Shu 写道: > > Dear All, > We would like to get a vote on this proposal. The

[DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-02 Thread Mohan Parthasarathy
Hi, I have created a proposal for adding some additional aggregation APIs like count. https://cwiki.apache.org/confluence/display/KAFKA/KIP-747+Add+support+for+basic+aggregation+APIs I have noted down some of the issues that need discussion. Thanks to Matthias for helping me with the scope of

[DISCUSS] KIP-748 Kafka Consumer Record Latency Metric

2021-06-02 Thread Ryan Dielhenn
Hey kafka-dev, I created KIP-748 as a proposal to add broker count metrics to the Quorum Controller. https://cwiki.apache.org/confluence/display/KAFKA/KIP-748%3A+Add+Broker+Count+Metrics#KIP748:AddBrokerCountMetrics Best, Ryan Dielhenn

Re: [VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Jason Gustafson
+1 overall. There is one complication which I think we may have to address in the implementation. The KIP documents an override to `fillInStackTrace`. Is that necessary? The trace would indeed be useful in this case because the new exception will wrap the exception raised from the Deserializer.

Re: [VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Sophie Blee-Goldman
Thanks for the KIP +1 (binding) - Sophie On Wed, Jun 2, 2021 at 3:43 PM Matthias J. Sax wrote: > +1 (binding) > > On 6/2/21 2:34 PM, Colin McCabe wrote: > > +1 (binding) > > > > Thanks, Sarwar. > > > > best, > > Colin > > > > On Wed, Jun 2, 2021, at 13:29, Sarwar Bhuiyan wrote: > >> Thanks

Re: [DISCUSS] KIP-726: Make the CooperativeStickyAssignor as the default assignor

2021-06-02 Thread Sophie Blee-Goldman
Hey Luke, It's been a while since the last update on this, which is mostly my fault for picking up other things in the meantime. I'm planning to get back to work on KAFKA-12477 next week but there are minimal changes to the current implementation given the proposal I put forth earlier in this KIP

[jira] [Created] (KAFKA-12882) Add RegisteredBrokerCount and UnfencedBrokerCount metrics to the QuorumController

2021-06-02 Thread Ryan Dielhenn (Jira)
Ryan Dielhenn created KAFKA-12882: - Summary: Add RegisteredBrokerCount and UnfencedBrokerCount metrics to the QuorumController Key: KAFKA-12882 URL: https://issues.apache.org/jira/browse/KAFKA-12882

Re: [VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Matthias J. Sax
+1 (binding) On 6/2/21 2:34 PM, Colin McCabe wrote: > +1 (binding) > > Thanks, Sarwar. > > best, > Colin > > On Wed, Jun 2, 2021, at 13:29, Sarwar Bhuiyan wrote: >> Thanks Colin, Matthias, and Jason on the discussion on this really old KIP. >> >> As discussed, I'd like to start to vote on

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #189

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 472007 lines...] [Pipeline] } [Pipeline] // withEnv [Pipeline] } [Pipeline] // withEnv [Pipeline] } [Pipeline] // withEnv [Pipeline] }

Re: [VOTING] KIP-693: Client-side Circuit Breaker for Partition Write Errors

2021-06-02 Thread Guozhang Wang
Thanks George, I'm generally +1 on the proposed change here, would like to review the detailed PR to see if there's any devils in the details. Guozhang On Tue, Jun 1, 2021 at 7:19 PM Guoqiang Shu wrote: > Dear All, > We would like to get a vote on this proposal. The implementation is linked

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-06-02 Thread Nakamura
Hi Colin, > Sure, we organize buffers by broker currently. However, we could set some maximum buffer size for records that haven't been assigned to a broker yet. OK, I think we're probably aligned then. I think we were using slightly different terminology (queue vs buffer) but we were actually

Re: [VOTE] KIP-738: Removal of Connect's internal converter properties

2021-06-02 Thread Randall Hauch
Thanks for putting this together, Chris. Technically, we don't need a new KIP to explicitly remove an API, config, etc. that was previously deprecated under an earlier approved KIP. But KIP-174 could have been a bit more explicit (note taken for future KIPs) that deprecation means "deprecation

[jira] [Resolved] (KAFKA-7815) SourceTask should expose ACK'd offsets, metadata

2021-06-02 Thread Ryanne Dolan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryanne Dolan resolved KAFKA-7815. - Fix Version/s: 2.4.0 Resolution: Fixed Equivalent functionality was included as part of

Re: [VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Colin McCabe
+1 (binding) Thanks, Sarwar. best, Colin On Wed, Jun 2, 2021, at 13:29, Sarwar Bhuiyan wrote: > Thanks Colin, Matthias, and Jason on the discussion on this really old KIP. > > As discussed, I'd like to start to vote on KIP-334: >

[VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Sarwar Bhuiyan
Thanks Colin, Matthias, and Jason on the discussion on this really old KIP. As discussed, I'd like to start to vote on KIP-334: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87297793 Thank you. Sarwar -- [image: Confluent] Sarwar Bhuiyan Staff

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #188

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 472872 lines...] [2021-06-02T20:21:34.888Z] [2021-06-02T20:21:34.888Z] MetricsTest > testControllerMetrics() STARTED

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Josep Prat
Done: https://issues.apache.org/jira/browse/KAFKA-12881 On Wed, Jun 2, 2021 at 9:54 PM Ismael Juma wrote: > Yeah, I think that would make sense regarding point 3. > > Ismael > > On Wed, Jun 2, 2021 at 12:44 PM Josep Prat > wrote: > > > Hi Ismael, > > > > That makes sense then. I guess the rest

[jira] [Created] (KAFKA-12881) Consider Un-Deprecation of Consumer#committed methods

2021-06-02 Thread Josep Prat (Jira)
Josep Prat created KAFKA-12881: -- Summary: Consider Un-Deprecation of Consumer#committed methods Key: KAFKA-12881 URL: https://issues.apache.org/jira/browse/KAFKA-12881 Project: Kafka Issue

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Ismael Juma
Yeah, I think that would make sense regarding point 3. Ismael On Wed, Jun 2, 2021 at 12:44 PM Josep Prat wrote: > Hi Ismael, > > That makes sense then. I guess the rest of the deprecated methods might > have similar issues like the ones you described. I see some will be covered > by

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Josep Prat
Hi Ismael, That makes sense then. I guess the rest of the deprecated methods might have similar issues like the ones you described. I see some will be covered by KAFKA-12588. Thanks for your time explaining the reasoning behind it doesn't make sense to remove them as of yet. Shall I create a

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Ismael Juma
Hi, 1. alterConfigs cannot be removed as there are some cases that are not possible via the incrementAlterConfigs API, there is a PR where a discussion is ongoing https://github.com/apache/kafka/pull/10514 2. Consumer.poll(long) has no direct replacement (Consumer.poll(Duration) has slightly

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Josep Prat
Hi Ismael, Thanks for your quick answer. For example, I've seen the org.apache.kafka.clients.consumer.Consumer#poll(long) deprecated in KIP-266 (Kafka 2.0.0) and in the same class methods org.apache.kafka.clients.consumer.Consumer#committed(org.apache.kafka.common.TopicPartition) and

Re: CLI command : Apache Kafka : to check LEADER

2021-06-02 Thread Girish L
Understood... Thank you very much Ran On Thu, 3 Jun, 2021, 12:31 am Ran Lupovich, wrote: > There is no one leader in kafka cluster for the a topic's partition, there > could be a different leader for each partition, > > > ./bin/kafka-topics --describe --topic topic-a --bootstrap-server >

Re: CLI command : Apache Kafka : to check LEADER

2021-06-02 Thread Ran Lupovich
There is no one leader in kafka cluster for the a topic's partition, there could be a different leader for each partition, ./bin/kafka-topics --describe --topic topic-a --bootstrap-server localhost:9092 Topic:topic-aPartitionCount:4ReplicationFactor:2 Configs: Topic:

Re: Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Ismael Juma
Hi Josep, We have removed nearly all of the apis we plan to remove for 3.0. The following JIRA is still in progress: https://issues.apache.org/jira/browse/KAFKA-12588 Aside from KAFKA-12588, which APIs were you thinking of that could be removed in 3.0? Ismael On Wed, Jun 2, 2021 at 11:52 AM

Removal of deprecated methods in Kafka Modules before 3.0.0

2021-06-02 Thread Josep Prat
Hi there, I was looking at the source code and I realized that there are several deprecated methods for long enough (2.5 or prior) that are not yet removed. Given that it has already been done for Kafka Streams, does it make sense if I would go through the other modules and create Jira tickets for

CLI command : Apache Kafka : to check LEADER

2021-06-02 Thread Girish L
hello all .. in apache kafka, could you please share the CLI command to check the Leader among the broker .. one command that can be run in the kafka broker cluster which gives the leader instance as output. Thanks

[jira] [Resolved] (KAFKA-12867) Trogdor ConsumeBenchWorker quits prematurely with maxMessages config

2021-06-02 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-12867. Fix Version/s: 3.0.0 Reviewer: Rajini Sivaram Resolution: Fixed > Trogdor

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-06-02 Thread Colin McCabe
On Tue, Jun 1, 2021, at 12:22, Nakamura wrote: > I think we're talking past each other a bit. I know about non-blocking > I/O. The problem I'm facing is how to preserve the existing semantics > without blocking. Right now callers assume their work is enqueued in-order > after

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-06-02 Thread Colin McCabe
On Tue, Jun 1, 2021, at 12:12, Ryanne Dolan wrote: > Colin, the issue for me isn't so much whether non-blocking I/O is used or > not, but the fact that the caller observes a long time between calling > send() and receiving the returned future. This behavior can be considered > "blocking" whether

[DISCUSS] KIP-746: Revise KRaft Metadata Records

2021-06-02 Thread Colin McCabe
Hi all, I have posted a KIP about updating the KRaft metadata records for 3.0. Check it out at : https://cwiki.apache.org/confluence/x/34zOCg best, Colin

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #187

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 472853 lines...] [2021-06-02T17:13:35.896Z] [2021-06-02T17:13:35.896Z] SaslPlainSslEndToEndAuthorizationTest >

Re: [DISCUSS] KIP-745: Connect API to restart connector and tasks

2021-06-02 Thread Ryanne Dolan
Thanks Randall, this would be great! Ryanne On Wed, Jun 2, 2021 at 11:40 AM Randall Hauch wrote: > Hello all, > > Many users struggle with the connector restart REST API only restarting the > Connector instance rather than everything they associated with a "named" > connector. I've created a

[DISCUSS] KIP-745: Connect API to restart connector and tasks

2021-06-02 Thread Randall Hauch
Hello all, Many users struggle with the connector restart REST API only restarting the Connector instance rather than everything they associated with a "named" connector. I've created a KIP that attempts to solve this problem via new query parameters on an existing REST API method, though by

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #186

2021-06-02 Thread Apache Jenkins Server
See

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

2021-06-02 Thread Rajini Sivaram
Hi Igor, If we want to support migration of server-side credentials using Admin API, we would need to get all of the data that is in the stored credential (salt, number of iterations and salted password). That is sufficient for a dictionary attack and to impersonate a server. Even though the salt

Re: [DISCUSS] KIP-618: Atomic commit of source connector records and offsets

2021-06-02 Thread Chris Egerton
Hi Tom, I do agree that it'd be safer to default to "required", but since at the time of the 3.0 release no existing connectors will have implemented the "SourceConnector::exactlyOnceSupport" method, it'd require all Connect users to downgrade to "requested" anyways in order to enable

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-06-02 Thread John Roesler
Thanks, Ivan! That sounds like a great plan to me. Two smaller KIPs are easier to agree on than one big one. I agree hopping and sliding windows will actually have a duplicating effect. We can avoid adding distinct() to the sliding window interface, but hopping windows are just a different

[jira] [Created] (KAFKA-12880) Remove deprecated Count and Rate in 3.0

2021-06-02 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12880: --- Summary: Remove deprecated Count and Rate in 3.0 Key: KAFKA-12880 URL: https://issues.apache.org/jira/browse/KAFKA-12880 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-618: Atomic commit of source connector records and offsets

2021-06-02 Thread Tom Bentley
Hi Chris, Just a minor question: I can see why the default for exactly.once.support is requested (you want a good first-run experience, I assume), but it's a little like engineering a safety catch and then not enabling it. Wouldn't it be safer to default to required, so that there's no way

[DISCUSS] KIP-714: Client metrics and observability

2021-06-02 Thread Magnus Edenhill
Hey all, I'm proposing KIP-714 to add remote Client metrics and observability. This functionality will allow centralized monitoring and troubleshooting of clients and their internals. Please see https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #185

2021-06-02 Thread Apache Jenkins Server
See Changes: -- [...truncated 406150 lines...] [2021-06-02T11:27:34.674Z] [2021-06-02T11:27:34.674Z] SaslPlainSslEndToEndAuthorizationTest >

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

2021-06-02 Thread Igor Soarez
Hi all, First of all, apologies for digging up this year-old thread. I believe that without further changes we will be losing support for a couple of important SCRAM management scenarios after the transition to a Zookeeper-less Kafka. One of the scenarios is a migration of a cluster. Topics

[jira] [Created] (KAFKA-12879) Compatibility break in Admin.listOffsets()

2021-06-02 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-12879: --- Summary: Compatibility break in Admin.listOffsets() Key: KAFKA-12879 URL: https://issues.apache.org/jira/browse/KAFKA-12879 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-12845) Rollback change which requires join key to be non null on KStream->GlobalKTable

2021-06-02 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-12845. - Resolution: Duplicate > Rollback change which requires join key to be non null on >