Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

2020-03-25 Thread Colin McCabe
Good news-- we are finally getting support for this in kafka-reassign-partitions.sh in Kafka 2.6. I updated the KIP to state that the controller changes were made in 2.4 and the command line changes will appear in 2.6. Previously it just said that the KIP was implemented in 2.4. Just as a qui

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-03-25 Thread Colin McCabe
Thanks, Kowshik, this looks good. In the "Schema" section, do we really need both __schema_version__ and __data_version__? Can we just have a single version field here? Shouldn't the Admin(Client) function have some way to get the min and max information that we're exposing as well? I guess w

Re: [DISCUSS] KIP-574: CLI Dynamic Configuration with file input

2020-03-25 Thread Colin McCabe
Hi Kamal, Are you suggesting that we not support STDIN here? I have mixed feelings. I think the ideal solution would be to support "-" in these tools whenever a file argument was expected. But that would be a bigger change than what we're talking about here. Maybe you are right and we should

Re: [DISCUSS] KIP-574: CLI Dynamic Configuration with file input

2020-03-25 Thread Colin McCabe
I agree with this. I think we'd be better off without the --delete-config-file option. It just seems confusing. best, Colin On Wed, Mar 25, 2020, at 03:48, Rajini Sivaram wrote: > Hi Aneel, > > Thanks for the KIP. As configurations get more complex, the ability to > provide compound configs i

[jira] [Created] (KAFKA-9768) rest.advertised.listener configuration is not handled properly by the worker

2020-03-25 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-9768: Summary: rest.advertised.listener configuration is not handled properly by the worker Key: KAFKA-9768 URL: https://issues.apache.org/jira/browse/KAFKA-9768 Project: K

Jenkins build is back to normal : kafka-2.4-jdk8 #177

2020-03-25 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-9767) Basic auth extension should have logging

2020-03-25 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-9767: Summary: Basic auth extension should have logging Key: KAFKA-9767 URL: https://issues.apache.org/jira/browse/KAFKA-9767 Project: Kafka Issue Type: Improvemen

Build failed in Jenkins: kafka-trunk-jdk11 #1288

2020-03-25 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9573: Fix JVM options to run early versions of Kafka on the latest -- [...truncated 2.96 MB...] org.apache.kafka.streams.internals.WindowSto

Build failed in Jenkins: kafka-2.1-jdk8 #257

2020-03-25 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8319: Make KafkaStreamsTest a non-integration test class (#7382) -- Started by an SCM change Running as SYSTEM [EnvInject] - Loading node environ

[jira] [Created] (KAFKA-9766) Suppress detailed responses for security-sensitive (PCI-DSS) environments

2020-03-25 Thread Connor Penhale (Jira)
Connor Penhale created KAFKA-9766: - Summary: Suppress detailed responses for security-sensitive (PCI-DSS) environments Key: KAFKA-9766 URL: https://issues.apache.org/jira/browse/KAFKA-9766 Project: Ka

Jenkins build is back to normal : kafka-trunk-jdk11 #1287

2020-03-25 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-580: Exponential Backoff for Kafka Clients

2020-03-25 Thread Sanjana Kaundinya
Hi Konstantine, Thanks for the feedback, I have addressed it on the [DISCUSS] thread and will update the KIP shortly. Thanks, Sanjana On Mar 25, 2020, 10:52 AM -0700, Konstantine Karantasis , wrote: > Hi Sanjana. > Thanks for the KIP! Seems quite useful not to overwhelm the brokers with > the d

Re: [DISCUSS] KIP-580: Exponential Backoff for Kafka Clients

2020-03-25 Thread Sanjana Kaundinya
Hi Konstantine, Thanks for the insightful feedback. I’ll address it here as well as update the KIP accordingly. I think it is important to call out the fact that we are leaving out Connect and Streams in the proposed changes, so that it can be addressed in future KIP/changes. As you pointed ou

Jenkins build is back to normal : kafka-trunk-jdk8 #4366

2020-03-25 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-9765) Could not add partitions to transaction due to errors

2020-03-25 Thread Prashant Waykar (Jira)
Prashant Waykar created KAFKA-9765: -- Summary: Could not add partitions to transaction due to errors Key: KAFKA-9765 URL: https://issues.apache.org/jira/browse/KAFKA-9765 Project: Kafka Issue

Build failed in Jenkins: kafka-2.5-jdk8 #78

2020-03-25 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-9758: Doc changes for KIP-523 and KIP-527 (#8343) -- [...truncated 2.90 MB...] org.apache.kafka.streams.TopologyTestDriverTest > shouldProcessF

[jira] [Created] (KAFKA-9764) Deprecate Stream Simple benchmark

2020-03-25 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9764: -- Summary: Deprecate Stream Simple benchmark Key: KAFKA-9764 URL: https://issues.apache.org/jira/browse/KAFKA-9764 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-9763) Recent changes to Connect's InsertField will fail to inject field on key of tombstone record

2020-03-25 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9763. -- Resolution: Duplicate Duplicate of KAFKA-9707, so closing this issue. > Recent changes to Conn

[jira] [Created] (KAFKA-9763) Recent changes to Connect's InsertField will fail to inject field on key of tombstone record

2020-03-25 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9763: Summary: Recent changes to Connect's InsertField will fail to inject field on key of tombstone record Key: KAFKA-9763 URL: https://issues.apache.org/jira/browse/KAFKA-9763

Re: [VOTE] KIP-580: Exponential Backoff for Kafka Clients

2020-03-25 Thread Konstantine Karantasis
Hi Sanjana. Thanks for the KIP! Seems quite useful not to overwhelm the brokers with the described requests from clients. You have the votes already, and I'm also in favor overall, but I've made a couple of questions (sorry for the delay) regarding Connect, which is also using retry.backoff.ms but

Re: [DISCUSS] KIP-580: Exponential Backoff for Kafka Clients

2020-03-25 Thread Konstantine Karantasis
Hi Sanjana and thanks for the KIP! Sorry for the late response, but I still have a few questions that you might find useful. The KIP currently does not mention Kafka Connect at all. I have read the discussion above where it'd been decided to leave Connect and Streams out of the proposed changes,

Re: [VOTE] KIP-519: Make SSL context/engine configuration extensible

2020-03-25 Thread Maulin Vasavada
bump On Wed, Mar 25, 2020 at 10:20 AM Maulin Vasavada wrote: > Hi all > > After much await on the approach conclusion we have a PR > https://github.com/apache/kafka/pull/8338. > > Can you please provide your vote so that we can more this forward? > > Thanks > Maulin > > On Sun, Jan 26, 2020 at 1

Re: [VOTE] KIP-519: Make SSL context/engine configuration extensible

2020-03-25 Thread Maulin Vasavada
Hi all After much await on the approach conclusion we have a PR https://github.com/apache/kafka/pull/8338. Can you please provide your vote so that we can more this forward? Thanks Maulin On Sun, Jan 26, 2020 at 11:03 PM Maulin Vasavada wrote: > Hi all > > After a good discussion on the KIP a

Re: [VOTE] KIP-570: Add leader epoch in StopReplicaRequest

2020-03-25 Thread Ismael Juma
Is it really true that the controller always sends two requests? Aren't the operations different (stop replica with delete versus stop replica without)? On Wed, Mar 25, 2020, 9:59 AM David Jacot wrote: > Hi all, > > I'd like to inform you that I have slightly changed the schema which was > propo

Re: [VOTE] KIP-570: Add leader epoch in StopReplicaRequest

2020-03-25 Thread David Jacot
Hi all, I'd like to inform you that I have slightly changed the schema which was proposed in the KIP. During the implementation, I have realized that the proposed schema did not work. The new one reorganises how topics/partitions are stored. I'd like to amend the current KIP with the following:

Build failed in Jenkins: kafka-2.5-jdk8 #77

2020-03-25 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-9677: Fix consumer fetch with small consume bandwidth quotas -- [...truncated 2.91 MB...] org.apache.kafka.streams.test.ConsumerRecordFact

[jira] [Resolved] (KAFKA-9758) Add documentations for KIP-523 and KIP-527

2020-03-25 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9758. Fix Version/s: 2.5.0 Resolution: Fixed > Add documentations for KIP-523 and KIP-527

Build failed in Jenkins: kafka-2.5-jdk8 #76

2020-03-25 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9752; New member timeout can leave group rebalance stuck (#8339) -- [...truncated 2.90 MB...] org.apache.kafka.streams.test.TestRecordTest > testM

[jira] [Created] (KAFKA-9762) Log Cleaner Stopped

2020-03-25 Thread jingcheng ma (Jira)
jingcheng ma created KAFKA-9762: --- Summary: Log Cleaner Stopped Key: KAFKA-9762 URL: https://issues.apache.org/jira/browse/KAFKA-9762 Project: Kafka Issue Type: Bug Components: log, lo

Build failed in Jenkins: kafka-2.4-jdk8 #176

2020-03-25 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9752; New member timeout can leave group rebalance stuck (#8339) -- [...truncated 2.75 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [DISCUSS] KIP-574: CLI Dynamic Configuration with file input

2020-03-25 Thread Rajini Sivaram
Hi Aneel, Thanks for the KIP. As configurations get more complex, the ability to provide compound configs in a file is really useful. I am not convinced about the `--delete-config-file` option though. I am not familiar with the Kubernetes case, but I guess if you create an entity with a file, it i

Jenkins build is back to normal : kafka-2.3-jdk8 #189

2020-03-25 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-2.5-jdk8 #75

2020-03-25 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-9700: Fix negative estimatedCompressionRatio (#8285) -- [...truncated 5.88 MB...] org.apache.kafka.streams.TestTopicsTest > testNonUsedOutputTopi

Build failed in Jenkins: kafka-2.4-jdk8 #175

2020-03-25 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9749; Transaction coordinator should treat KAFKA_STORAGE_ERROR as -- [...truncated 7.84 MB...] org.apache.kafka.streams.kstream.internals.KStreamK

Build failed in Jenkins: kafka-trunk-jdk11 #1286

2020-03-25 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9700: Fix negative estimatedCompressionRatio (#8285) [github] HOTFIX: fix log message in version probing system test (#8341) [github] KAFKA-9752; New member timeout can leave group

Re: [DISCUSS] KIP-574: CLI Dynamic Configuration with file input

2020-03-25 Thread Kamal Chandraprakash
STDIN wasn't standard practice in other scripts like kafka-console-consumer.sh, kafka-console-producer.sh and kafka-acls.sh in which the props file is accepted via consumer.config / producer.config / command-config parameter. Shouldn't we have to maintain the uniformity across scripts? On Mon, Ma

[jira] [Created] (KAFKA-9761) kafka-consumer-groups tool overrides admin client defaults with a short 5 s timeout

2020-03-25 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-9761: --- Summary: kafka-consumer-groups tool overrides admin client defaults with a short 5 s timeout Key: KAFKA-9761 URL: https://issues.apache.org/jira/browse/KAFKA-9761 Proje

Build failed in Jenkins: kafka-trunk-jdk8 #4365

2020-03-25 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9700: Fix negative estimatedCompressionRatio (#8285) [github] HOTFIX: fix log message in version probing system test (#8341) [github] KAFKA-9752; New member timeout can leave group r

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-03-25 Thread Kowshik Prakasam
Hi Boyang, Great catch, thanks! I have fixed this now. Please have a look, and let me if you have any questions. Cheers, Kowshik On Tue, Mar 24, 2020 at 11:06 PM Boyang Chen wrote: > Nice KIP Kowshik! This is a long due feature for the ease of both client > side and server side upgrade in gen