Re: [VOTE] KIP-280: Enhanced log compaction

2019-11-06 Thread Matthias J. Sax
+1 (binding) On 11/5/19 11:44 AM, Senthilnathan Muthusamy wrote: > Thanks Gouzhang and I have made a note in the JIRA item to update the wiki. > > Till now got 1 +1 binding... waiting for 2 more +1 binding... thnx! > > Regards, > Senthil > > -Original Message- > From: Guozhang Wang >

[jira] [Created] (KAFKA-9150) DescribeGroup uses member assignment as metadata

2019-11-06 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-9150: -- Summary: DescribeGroup uses member assignment as metadata Key: KAFKA-9150 URL: https://issues.apache.org/jira/browse/KAFKA-9150 Project: Kafka Issue Type

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

2019-11-06 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: remove reference to unused Assignment error code (#7645) -- [...truncated 5.03 MB...] kafka.api.GroupEndToEndAuthorizationTest > testProduceCo

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

2019-11-06 Thread Apache Jenkins Server
See Changes: [cmccabe] MINOR: Rework NewPartitionReassignment public API (#7638) -- [...truncated 8.29 MB...] org.apache.kafka.streams.test.OutputVerifierTest > shouldF

[jira] [Created] (KAFKA-9151) KafkaProducer.send should warmup metadata information when application starts

2019-11-06 Thread Tao Chen (Jira)
Tao Chen created KAFKA-9151: --- Summary: KafkaProducer.send should warmup metadata information when application starts Key: KAFKA-9151 URL: https://issues.apache.org/jira/browse/KAFKA-9151 Project: Kafka

[jira] [Created] (KAFKA-9152) Improve Sensor Retrieval

2019-11-06 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9152: Summary: Improve Sensor Retrieval Key: KAFKA-9152 URL: https://issues.apache.org/jira/browse/KAFKA-9152 Project: Kafka Issue Type: Improvement Com

[jira] [Resolved] (KAFKA-9079) System Test Failure: TransactionsTest

2019-11-06 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9079?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-9079. -- Fix Version/s: (was: 2.5.0) 2.4.0 Resolution: Fixed Issue resolved by pul

Re: [DISCUSS] KIP-409: Allow creating under-replicated topics and partitions

2019-11-06 Thread Mickael Maison
Hi, Thanks Colin for the feedback. Edo and I have updated the KIP accordingly. Can you take another look? On Tue, Oct 22, 2019 at 12:20 AM Colin McCabe wrote: > > Hi Mickael, > > We don't have any official way for brokers to join the cluster other than > showing up and registering themselves i

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

2019-11-06 Thread Satish Duggana
Hi Jun, >21. Could you elaborate a bit why the positions in remote segment is different from the local one? I thought that they are identical copies. They may not always be the same. Let me take an example here. If remote storage is enabled with encryption then those local positions may not be th

RE: [VOTE] KIP-280: Enhanced log compaction

2019-11-06 Thread Senthilnathan Muthusamy
Thanks Matthias! Received 2 +1 binding... looking for one more +1 binding ! Regards, Senthil -Original Message- From: Matthias J. Sax Sent: Wednesday, November 6, 2019 12:10 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-280: Enhanced log compaction +1 (binding) On 11/5/19 11:4

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-11-06 Thread Viktor Somogyi-Vass
Hi Elmahdi, I've added the JIRA to the KIP (and also below) where you can track the progress (but more subtask will come as the current ones don't represent the full work to be done). https://issues.apache.org/jira/browse/KAFKA-9119 Viktor On Tue, Nov 5, 2019 at 5:15 PM Elmahdi FRID wrote: > H

[jira] [Created] (KAFKA-9153) Kafka brokers randomly crash (SIGSEGV due to kafka errors)

2019-11-06 Thread Tristan (Jira)
Tristan created KAFKA-9153: -- Summary: Kafka brokers randomly crash (SIGSEGV due to kafka errors) Key: KAFKA-9153 URL: https://issues.apache.org/jira/browse/KAFKA-9153 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-9154) ProducerId generation should be managed by the Controller

2019-11-06 Thread Viktor Somogyi-Vass (Jira)
Viktor Somogyi-Vass created KAFKA-9154: -- Summary: ProducerId generation should be managed by the Controller Key: KAFKA-9154 URL: https://issues.apache.org/jira/browse/KAFKA-9154 Project: Kafka

Re: [DISCUSS] KIP-501 Avoid out-of-sync or offline partitions when follower fetch requests not processed in time

2019-11-06 Thread Satish Duggana
Hi Dhruvil, Thanks for looking into the KIP. 10. I have an initial sketch of the KIP-500 in commit[a] which discusses tracking the pending fetch requests. Tracking is not done in Partition#readRecords because if it takes longer in reading any of the partitions then we do not want any of the replic

Re: [VOTE] KIP-150 - Kafka-Streams Cogroup

2019-11-06 Thread Bill Bejeck
Thanks for the KIP. +1 (binding) -Bill On Wed, Nov 6, 2019 at 1:22 AM Matthias J. Sax wrote: > +1 (binding) > > > On 10/31/19 10:52 AM, Walker Carlson wrote: > > Hello all, > > > > I'd like to call a vote on the updated KIP-150: Kafka-Streams Cogroup > > found here > > < > https://cwiki.apache

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

2019-11-06 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-9079: Fix reset logic in transactional message copier -- [...truncated 5.08 MB...] kafka.server.KafkaConfigTest > testLogRollTimeNoConfigProvid

Re: [DISCUSS] KIP-513: Distinguish between Key and Value serdes in scala wrapper library for kafka streams

2019-11-06 Thread John Roesler
Hey Mykhailo, I just wanted to let you know that I'm looking at your proposal, but it'll take me a little while to re-activate the Scala part of my brain. For everyone's benefit, here's the link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-513%3A+Distinguish+between+Key+and+V

gradle uploadArchives

2019-11-06 Thread Carl Graving
I was trying to use this gradle task and it works, but the files are all having a timestamp appended to the version. Places like maven central don't have these timestamps. Is there an option to not have the timestamps appended when running this task? Thanks, Carl

Re: [VOTE] KIP-150 - Kafka-Streams Cogroup

2019-11-06 Thread Guozhang Wang
+1 (binding), Thanks Walker! Guozhang On Wed, Nov 6, 2019 at 8:41 AM Bill Bejeck wrote: > Thanks for the KIP. > +1 (binding) > > -Bill > > > On Wed, Nov 6, 2019 at 1:22 AM Matthias J. Sax > wrote: > > > +1 (binding) > > > > > > On 10/31/19 10:52 AM, Walker Carlson wrote: > > > Hello all, > >

[jira] [Resolved] (KAFKA-9140) Consumer gets stuck rejoining the group indefinitely

2019-11-06 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-9140. -- Assignee: Guozhang Wang Resolution: Fixed > Consumer gets stuck rejoining the group indefinitely

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

2019-11-06 Thread Apache Jenkins Server
See

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

2019-11-06 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-150 - Kafka-Streams Cogroup

2019-11-06 Thread Walker Carlson
Thanks everyone for the votes! The KIP has been accepted, with 3 binding votes from Matthias, Bill and Guozhang. Walker On Wed, Nov 6, 2019 at 9:41 AM Guozhang Wang wrote: > +1 (binding), > > Thanks Walker! > > Guozhang > > On Wed, Nov 6, 2019 at 8:41 AM Bill Bejeck wrote: > > > Thanks for th

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-06 Thread Vinoth Chandar
+1 to John, suggestion on Duration/Instant and dropping the API to fetch all store's lags. However, I do think we need to return lags per topic partition. So not sure if single return value would work? We need some new class that holds a TopicPartition and Duration/Instant variables together? 10)

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

2019-11-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-9140: Also reset join future when generation was reset in order to -- [...truncated 2.70 MB...] org.apache.kafka.streams.internals.KeyValueSto

Re: [DISCUSS] KIP-513: Distinguish between Key and Value serdes in scala wrapper library for kafka streams

2019-11-06 Thread John Roesler
Hi Mykhailo, I've been mulling over your KIP today. I think that what you're proposing makes sense. I'm having a little trouble wrapping my head around the exact problem with the current API, though... It sounds like you're saying that you want to have different key and value serdes for the same

Re: [DISCUSS] KIP-544: Make metrics exposed via JMX configurable

2019-11-06 Thread Xavier Léauté
> > Since these configs will work with Kafka's own metrics library, will the > configs be part of the clients' configurations? It would be good to point > that out explicitly in the KIP. > Those configs are currently only at the broker level. If we feel this is useful on the client as well, we cou

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

2019-11-06 Thread Ying Zheng
21. I am not sure that I understood the need for RemoteLogIndexEntry and its relationship with RemoteLogSegmentInfo. It seems that RemoteLogIndexEntry are offset index entries pointing to record batches inside a segment. That seems to be the same as the .index file? We do not assume the how the da

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

2019-11-06 Thread Ying Zheng
On Wed, Nov 6, 2019 at 4:33 PM Ying Zheng wrote: > 21. I am not sure that I understood the need for RemoteLogIndexEntry and > its relationship with RemoteLogSegmentInfo. It seems > that RemoteLogIndexEntry are offset index entries pointing to record > batches inside a segment. That seems to be th

Re: Subject: [VOTE] 2.2.2 RC2

2019-11-06 Thread Eric Lalonde
Hello, In an effort to assist in the verification of release candidates, I have authored the following quick-and-dirty utility to help people verify release candidate artifacts: https://github.com/elalonde/kafka/blob/master/bin/verify-kafka-rc.sh

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

2019-11-06 Thread Satish Duggana
>Depends on the implementation, the data of one segment may not necessary be stored in a single file. There could be a maximum object / chunk / file size restriction on the remote storage. So, one Kafka segment could be saved in multiple chunks in remote storage. Having one local segment can be st

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

2019-11-06 Thread Tom Bentley
Hi Ying, Because only inactive segments can be shipped to remote storage, to be able > to ship log data as soon > as possible, we will roll log segment very fast (e.g. every half hour). > So that means a consumer which gets behind by half an hour will find its reads being served from remote stora

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

2019-11-06 Thread Satish Duggana
>So that means a consumer which gets behind by half an hour will find its reads being served from remote storage. And, if I understand the proposed algorithm, each such consumer fetch request could result in a separate fetch request from the remote storage. I.e. there's no mechanism to amortize the

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

2019-11-06 Thread Satish Duggana
Hi Tom,' Sorry, I missed the other question. >(Actually the doc for RemoteStorageManager.read() says "It will read at least one batch, if the 1st batch size is larger than maxBytes.". Does that mean the broker might have to retry with increased maxBytes if the first request fails to read a batch?

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

2019-11-06 Thread Tom Bentley
Hi Satish, >So that means a consumer which gets behind by half an hour will find its > reads being served from remote storage. And, if I understand the proposed > algorithm, each such consumer fetch request could result in a separate > fetch request from the remote storage. I.e. there's no mechani

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

2019-11-06 Thread Satish Duggana
>>Depends on the implementation, the data of one segment may not necessary be stored in a single file. There could be a maximum object / chunk / file size restriction on the remote storage. So, one Kafka segment could be saved in multiple chunks in remote storage. >Having one local segment can be

Re: [DISCUSS] KIP-535: Allow state stores to serve stale reads during rebalance

2019-11-06 Thread Navinder Brar
+1 on implementing offset based lag for now and push time-based lag to a later point in time when broker changes are done. Although time-based lag enhances the readability, it would not be a make or break change for implementing this KIP.  Vinoth has explained the role of KeyQueryMetadata, let

[jira] [Resolved] (KAFKA-9150) DescribeGroup uses member assignment as metadata

2019-11-06 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-9150. -- Resolution: Fixed > DescribeGroup uses member assignment as metadata >

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

2019-11-06 Thread Apache Jenkins Server
See Changes: [bbejeck] [MINOR] Clean up PartitionAssignor for KIP-441 (#7649) [wangguoz] KAFKA-8729: Change `PartitionResponse` to include all troubling records -- [...