Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-02-06 Thread Colin McCabe
Hi Becket, I would argue that using IDs for partitions is not a performance improvement, but actually a completely different way of accomplishing what this KIP is trying to solve. If you give partitions globally unique IDs, and use a different ID when re-creating a topic partition, you don't

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-06 Thread Colin McCabe
On Wed, Jan 24, 2018, at 15:59, Jun Rao wrote: > Hi, Litao, > > Thanks for the KIP. Good proposal. A few comments below. > > 1. The KIP says "select the live replica with the largest LEO". I guess > what you meant is selecting the live replicas in ISR with the largest LEO? > > 2. I agree that

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-02-06 Thread Colin McCabe
Hi Dong & Guozhang, Thanks... this is a really interesting discussion! My first thought is, why should the controller be involved in choosing the leader of the quorum? That seems like an internal quorum issue. All of the state that is needed to select the proper leader is on the nodes

Re: Query on min.insync.replicas property

2018-02-06 Thread Ted Yu
That should include the leader node. On Tue, Feb 6, 2018 at 8:46 PM, rishi jain wrote: > Hi, > > I have an doubt on min.insync.replicas property in kafka server.properties > files. > If we set any value for "min.insync.replicas" property, is that including > leader node

Query on min.insync.replicas property

2018-02-06 Thread rishi jain
Hi, I have an doubt on min.insync.replicas property in kafka server.properties files. If we set any value for "min.insync.replicas" property, is that including leader node or excluding leader node? Thanks Rishabh Jain

[jira] [Created] (KAFKA-6539) KafkaConsumer endlessly try to connect to a dead broker, ignoring brokers alive

2018-02-06 Thread Song Younghwan (JIRA)
Song Younghwan created KAFKA-6539: - Summary: KafkaConsumer endlessly try to connect to a dead broker, ignoring brokers alive Key: KAFKA-6539 URL: https://issues.apache.org/jira/browse/KAFKA-6539

Re: [DISCUSS] KIP-251: Allow timestamp manipulation in Processor API

2018-02-06 Thread Matthias J. Sax
I am open to remove the index API and also open to piggyback this change on this KIP... Not sure if this was the intention of your comment? Or should be have a separate KIP for this? Not sure, what other think about removing the index API? -Matthias On 2/5/18 4:13 PM, Guozhang Wang wrote: >

Re: Documentation build system

2018-02-06 Thread Guozhang Wang
Ewen, Thanks for re-picking this up again. I'm big +1 as I was two years ago :P One thing that may still worth pointing out is that migrating from html means that for any edits it would require one more compilation step to review / compare the diffs if it is not only wording but also formatting /

Documentation build system

2018-02-06 Thread Ewen Cheslack-Postava
Hi all, I just wrote a note in https://issues.apache.org/jira/browse/KAFKA-2967 with a proposal for changing how docs are written. I want to move on this soon if possible and normally would just leave the discussion to the JIRA, but as I think this is something everyone has an opinion on and

Jenkins build is back to normal : kafka-trunk-jdk9 #373

2018-02-06 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk7 #3160

2018-02-06 Thread Apache Jenkins Server
See Changes: [junrao] Minor: fix compilation error in KAFKA-6254 -- [...truncated 412.82 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[jira] [Created] (KAFKA-6538) Enhance ByteStore exceptions with more context information

2018-02-06 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6538: -- Summary: Enhance ByteStore exceptions with more context information Key: KAFKA-6538 URL: https://issues.apache.org/jira/browse/KAFKA-6538 Project: Kafka

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

2018-02-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-4750: Bypass null value and treat it as deletes (#4508) [junrao] Minor: fix compilation error in KAFKA-6254 -- [...truncated 413.67 KB...]

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2018-02-06 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I have add some changes to the KIP based on the Pull Request: https://github.com/apache/kafka/pull/4454#issuecomment-360553277 : * Reduce the scope of the operations to Consumer Groups to avoid complexity of making assignments generic for Consumer and Connect groups. If Connect

Re: [VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2018-02-06 Thread Jorge Esteban Quilcate Otoya
Thanks Matthias. I have updated the version on KIP main page also. There are some changes that arise on the Pull Request. I will comment them on the discussion thread. Cheers, Jorge. El vie., 2 feb. 2018 a las 20:50, Matthias J. Sax () escribió: > Feature freeze for 1.1

Build failed in Jenkins: kafka-trunk-jdk9 #372

2018-02-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-4750: Bypass null value and treat it as deletes (#4508) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building

Build failed in Jenkins: kafka-trunk-jdk7 #3159

2018-02-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-4750: Bypass null value and treat it as deletes (#4508) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building

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

2018-02-06 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-6184; report a metric of the lag between the consumer offset ... [wangguoz] MINOR: exchange redundant Collections.addAll with parameterized

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

2018-02-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6504: Ensure uniqueness of connect task metric sensor creation [me] MINOR: Update release script with new remote, better error handling, --

Build failed in Jenkins: kafka-trunk-jdk9 #371

2018-02-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: exchange redundant Collections.addAll with parameterized -- Started by an SCM change [EnvInject] - Loading node environment variables.

Build failed in Jenkins: kafka-trunk-jdk7 #3158

2018-02-06 Thread Apache Jenkins Server
See Changes: [me] MINOR: Update release script with new remote, better error handling, [junrao] KAFKA-6184; report a metric of the lag between the consumer offset ... [wangguoz] MINOR: exchange redundant

Build failed in Jenkins: kafka-trunk-jdk9 #370

2018-02-06 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-6184; report a metric of the lag between the consumer offset ... -- Started by an SCM change [EnvInject] - Loading node environment variables.

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-02-06 Thread Dong Lin
Hey Jun, Interestingly I am also planning to sketch a KIP to allow partition expansion for keyed topics after this KIP. Since you are already doing that, I guess I will just share my high level idea here in case it is helpful. The motivation for the KIP is that we currently lose order guarantee

Jenkins build is back to normal : kafka-1.1-jdk7 #16

2018-02-06 Thread Apache Jenkins Server
See

[jira] [Reopened] (KAFKA-5889) MetricsTest is flaky

2018-02-06 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5889?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu reopened KAFKA-5889: --- As of 332e698ac9c74ce29317021b03a54512c92ac8b3 , I got: {code} kafka.metrics.MetricsTest > testMetricsLeak FAILED

[jira] [Resolved] (KAFKA-6184) report a metric of the lag between the consumer offset and the start offset of the log

2018-02-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6184. Resolution: Fixed Fix Version/s: 1.2.0 The PR is merged to trunk. > report a metric of the lag

Re: [VOTE] KIP-232: Detect outdated metadata using leaderEpoch and partitionEpoch

2018-02-06 Thread Jun Rao
Hi, Dong, Thanks for the KIP. It looks good overall. We are working on a separate KIP for adding partitions while preserving the ordering guarantees. That may require another flavor of partition epoch. It's not very clear whether that partition epoch can be merged with the partition epoch in this

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

2018-02-06 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6537) Duplicate consumers after consumer group rebalance

2018-02-06 Thread Michael Golovanov (JIRA)
Michael Golovanov created KAFKA-6537: Summary: Duplicate consumers after consumer group rebalance Key: KAFKA-6537 URL: https://issues.apache.org/jira/browse/KAFKA-6537 Project: Kafka

Re: [DISCUSS] KIP-248 Create New ConfigCommand That Uses The New AdminClient

2018-02-06 Thread Rajini Sivaram
Hi Viktor, While implementing KAFKA-6494, I realised that there is a mismatch between the --alter command of ConfigCommand and AlterConfigs request. ConfigCommand uses --add-config and --delete-config to make incremental updates. --add-config reads all the configs from ZooKeeper and adds the

Jenkins build is back to normal : kafka-trunk-jdk7 #3156

2018-02-06 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-1.1-jdk7 #15

2018-02-06 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6528: Fix transient test failure in testThreadPoolResize (#4526) -- [...truncated 410.56 KB...] kafka.log.LogCleanerIntegrationTest >

Re: [VOTE] 1.0.1 RC0

2018-02-06 Thread Ted Yu
I see maven artifacts now. Thanks, Ewen. On Mon, Feb 5, 2018 at 10:35 PM, Ewen Cheslack-Postava wrote: > @Ted Check now. I went back and double checked the details. Seems like the > release details may have changed in a way that was not clearly documented > because we have

[jira] [Resolved] (KAFKA-6528) Transient failure in DynamicBrokerReconfigurationTest.testThreadPoolResize

2018-02-06 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-6528. --- Resolution: Fixed Fix Version/s: 1.1.0 > Transient failure in