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

2019-11-05 Thread Matthias J. Sax
Thanks for updating the KIP, Senthil. @Eric: good point about using the last found header for the key instead of the first! I don't have any further comments at this point. -Matthias On 11/5/19 11:37 AM, Senthilnathan Muthusamy wrote: > Hi Guozhang, > > Sure and I have made a note in the

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

2019-11-05 Thread Apache Jenkins Server
See Changes: [cmccabe] KAFKA-9137: Fix incorrect FetchSessionCache eviction logic (#7640) [wangguoz] HOTFIX: remove reference to unused Assignment error code (#7645) --

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

2019-11-05 Thread Matthias J. Sax
Navinder, thanks for updating the KIP. Couple of follow up questions: (10) Why do we need to introduce the class `KeyQueryMetadata`? (20) Why do we introduce the two methods `allMetadataForKey()`? Would it not be simpler to add `Collection standbyMetadataForKey(...)`. This would align with new

[jira] [Created] (KAFKA-9149) Avoid temp byte array creation when use ByteBufferSerializer

2019-11-05 Thread chenxu (Jira)
chenxu created KAFKA-9149: - Summary: Avoid temp byte array creation when use ByteBufferSerializer Key: KAFKA-9149 URL: https://issues.apache.org/jira/browse/KAFKA-9149 Project: Kafka Issue Type:

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

2019-11-05 Thread Matthias J. Sax
+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 > > > Thanks, > Walker > signature.asc

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

2019-11-05 Thread Navinder Brar
Thanks John and Vinoth for converging thoughts on AssignmentInfo. - Report the time difference between the last consumed changelog record's timestamp and the changelog tail record's timestamp. This is an indicator of how fresh the local copy of a store is with respect to the active copy.

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

2019-11-05 Thread Apache Jenkins Server
See

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

2019-11-05 Thread John Roesler
Hey Vinoth, Really sorry, I just remembered that I started a reply earlier today, but got side-tracked. Regarding the AssignmentInfo extension: Your explanation for this point makes sense. I was incorrectly thinking that the cluster metadata was shared with all members, but now I see it's only

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

2019-11-05 Thread Vinoth Chandar
Ping :) Any thoughts? On Mon, Nov 4, 2019 at 5:13 PM Vinoth Chandar wrote: > >> I'm having some trouble wrapping my head around what race conditions > might occur, other than the fundamentally broken state in which different > instances are running totally different topologies. > 3. @both

[jira] [Created] (KAFKA-9148) Consider forking RocksDB for Streams

2019-11-05 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9148: -- Summary: Consider forking RocksDB for Streams Key: KAFKA-9148 URL: https://issues.apache.org/jira/browse/KAFKA-9148 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-9147) zookeeper service not running

2019-11-05 Thread parimal (Jira)
parimal created KAFKA-9147: -- Summary: zookeeper service not running Key: KAFKA-9147 URL: https://issues.apache.org/jira/browse/KAFKA-9147 Project: Kafka Issue Type: Test Affects Versions:

[jira] [Resolved] (KAFKA-8367) Non-heap memory leak in Kafka Streams

2019-11-05 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8367?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-8367. Resolution: Fixed > Non-heap memory leak in Kafka Streams >

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2019-11-05 Thread Sophie Blee-Goldman
> Personally, I think Matthias’s concern is valid, but on the other hand Kafka Streams has already > optimizer in place which alters topology independently from user I agree (with you) and think this is a good way to put it -- we currently auto-repartition for the user so that they don't have to

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

2019-11-05 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9110: Improve efficiency of disk reads when TLS is enabled (#7604) -- [...truncated 8.30 MB...]

[jira] [Created] (KAFKA-9146) Add option to force delete members in stream reset tool

2019-11-05 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9146: -- Summary: Add option to force delete members in stream reset tool Key: KAFKA-9146 URL: https://issues.apache.org/jira/browse/KAFKA-9146 Project: Kafka Issue

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

2019-11-05 Thread Apache Jenkins Server
See Changes: [cmccabe] MINOR: Rework NewPartitionReassignment public API (#7638) -- [...truncated 2.69 MB...] org.apache.kafka.streams.MockTimeTest > shouldSetStartTime

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2019-11-05 Thread Levani Kokhreidze
Hello all, While https://github.com/apache/kafka/pull/7170 is under review and it’s almost done, I want to resurrect discussion about this KIP to address couple of concerns raised by Matthias and John. As a reminder, idea of the KIP-221 was to allow

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

2019-11-05 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9110: Improve efficiency of disk reads when TLS is enabled (#7604) -- [...truncated 4.56 MB...] kafka.api.PlaintextConsumerTest >

Re: [DISCUSS] KIP-536: Propagate broker timestamp to Admin API

2019-11-05 Thread Noa Resare
I agree with that. And looking at the MetadataResponse fields it seems there has been some feature creep already here. Does the client use rack information, for example? I guess one could do this either by introducing a new leaner message pair, used specifically enable client operation, and

[jira] [Created] (KAFKA-9145) AbstractCoordinator should respect retry backoff between rebalances

2019-11-05 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-9145: -- Summary: AbstractCoordinator should respect retry backoff between rebalances Key: KAFKA-9145 URL: https://issues.apache.org/jira/browse/KAFKA-9145 Project: Kafka

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

2019-11-05 Thread Senthilnathan Muthusamy
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 Sent: Monday, November 4, 2019 11:01 AM To: dev Subject: Re: [VOTE] KIP-280:

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

2019-11-05 Thread Senthilnathan Muthusamy
Hi Guozhang, Sure and I have made a note in the JIRA item to make sure the wiki is updated. Thanks, Senthil -Original Message- From: Guozhang Wang Sent: Monday, November 4, 2019 11:00 AM To: dev Subject: Re: [DISCUSS] KIP-280: Enhanced log compaction Hello Senthilnathan, Thanks for

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

2019-11-05 Thread Senthilnathan Muthusamy
Thanks for pointing it out Eric. Updated the KIP... Regards, Senthil -Original Message- From: Guozhang Wang Sent: Monday, November 4, 2019 11:52 AM To: dev Subject: Re: [DISCUSS] KIP-280: Enhanced log compaction Eric, I think that's a good point, in `Headers.java` we also designed

[jira] [Created] (KAFKA-9144) Early expiration of producer state can cause coordinator epoch to regress

2019-11-05 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-9144: -- Summary: Early expiration of producer state can cause coordinator epoch to regress Key: KAFKA-9144 URL: https://issues.apache.org/jira/browse/KAFKA-9144 Project:

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

2019-11-05 Thread Elmahdi FRID
Hello Folks any status abbout this kip and it's possible to test this use case ? On 2019/08/01 21:04:46, "Colin McCabe" wrote: > Hi all, > > I've written a KIP about removing ZooKeeper from Kafka. Please take a look > and let me know what you think: > >

[jira] [Created] (KAFKA-9143) DistributedHerder misleadingly log error on connector task reconfiguration

2019-11-05 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-9143: - Summary: DistributedHerder misleadingly log error on connector task reconfiguration Key: KAFKA-9143 URL: https://issues.apache.org/jira/browse/KAFKA-9143 Project:

Re: Subject: [VOTE] 2.2.2 RC2

2019-11-05 Thread Randall Hauch
Thanks, Mickael! Anyone else get a chance to validate the 2.2.2 RC2 build? It'd be great to get this out the door. Randall On Tue, Nov 5, 2019 at 6:34 AM Mickael Maison wrote: > +1 (non binding) > I verified signatures, built it from source, ran unit tests and quickstart > > > > On Fri, Oct

Re: [DISCUSS] KIP-536: Propagate broker timestamp to Admin API

2019-11-05 Thread Gwen Shapira
It isn't just about saving space. It increases complexity to default to always sharing a bit of information that is really only needed in a single use-case. We avoid doing this as a matter of good protocol design. Arguably, this should not really piggyback on cluster metadata at all, since the

Re: [DISCUSS] KIP-536: Propagate broker timestamp to Admin API

2019-11-05 Thread Noa Resare
It would certainly be possible to have the field be optional and only include it if some flag is set in the DescribeClusterOptions instance passed to Admin.describeCluster() that in turn would translate to a boolean in MetadataRequest indicating that we are asking for this piece of information.

Re: Subject: [VOTE] 2.2.2 RC2

2019-11-05 Thread Mickael Maison
+1 (non binding) I verified signatures, built it from source, ran unit tests and quickstart On Fri, Oct 25, 2019 at 3:10 PM Randall Hauch wrote: > > Hello all, we identified around three dozen bug fixes, including an update > of a third party dependency, and wanted to release a patch release

Re: [VOTE] KIP-523 Add KStream#toTable to the Streams DSL

2019-11-05 Thread aishwarya kumar
Thank you so much for the votes. I will consider that the KIP is accepted, with 3 binding votes from Matthias, Bill and Guozhang. And 1 non-binding vote from John. Best, Aishwarya On Mon, Nov 4, 2019, 12:18 PM Guozhang Wang wrote: > +1 (binding), thanks Aishwarya! > > On Sun, Nov 3, 2019

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

2019-11-05 Thread Viktor Somogyi-Vass
Hi Xavier, That's certainly an option, however it does not solve the problem for our users that still rely on JMX integration to collect metrics. Absolutely. We already provide the ability to write reporter plugins via the MetricsReporter interface. And rather than building a separate HTTP