Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-09-06 Thread Tom Bentley
Hi Ted and Colin, Thanks for the comments. It seems you're both happier with reassign rather than assign, so I'm happy to stick with that. On 5 September 2017 at 18:46, Colin McCabe wrote: > ... > Do we expect that reducing the number of partitions will ever be > supported by this API? It

[GitHub] kafka pull request #3799: KAFKA-5597 [WIP] Alternate way to do Metrics docs ...

2017-09-06 Thread wushujames
GitHub user wushujames opened a pull request: https://github.com/apache/kafka/pull/3799 KAFKA-5597 [WIP] Alternate way to do Metrics docs generation I was about to start on the next round of autogeneration of metrics docs, but I wanted to @guozhangwang 's opinion on this first. This

[GitHub] kafka pull request #3800: KAFKA-5764: Add toLowerCase support to sasl.kerber...

2017-09-06 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/3800 KAFKA-5764: Add toLowerCase support to sasl.kerberos.principal.to.local rule You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka K

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-09-06 Thread Tom Bentley
Hi Colin, Thanks for taking the time to respond. On 5 September 2017 at 22:22, Colin McCabe wrote: > ... > Why does there need to be a map at all in the API? >From a purely technical PoV there doesn't, but doing something else would make the API inconsistent with other similar AdminClient *Re

Re: [VOTE] KIP-138: Change punctuate semantics

2017-09-06 Thread Michael Noll
Thanks, Guozhang. FWIW here's my (non-binding) +1. Sorry for the delay, Gmail decided to mark this thread as spam :roll-eyes:. -Michael On Tue, Sep 5, 2017 at 10:36 PM, Guozhang Wang wrote: > Thanks for your inputs. The main motivation is indeed to achieve > consistency as we use "wall-cloc

Re: [VOTE] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-09-06 Thread Attila Kreiner
Hi All, No more votes received for this, so the final decision is: accepted. Thx, Attila 2017-09-02 6:57 GMT+02:00 Attila Kreiner : > Hi Jason, > > Sorry, I didn't know that one. Then I assume we should wait a few days > before moving forward. > > Regards, > Attila > > 2017-09-01 18:01 GMT+02:0

Fw: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Paolo Patierno
Hi devs, I haven't seen any votes for this since last month. Is there something that should be addressed in the KIP (it didn't have any comments anymore and for this reason I started the vote). Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded

Re: [VOTE] KIP-188 - Add new metrics to support health checks

2017-09-06 Thread Rajini Sivaram
Ok, thanks, leaving as is. On Wed, Sep 6, 2017 at 1:22 AM, Jason Gustafson wrote: > > > > I think I prefer the names with `Message` in them. For people less > familiar > > with Kafka, it makes it a bit clearer, I think. > > > Works for me. > > On Tue, Sep 5, 2017 at 5:19 PM, Ismael Juma wrote:

[GitHub] kafka pull request #3772: KAFKA-5817: Add Serialized class and overloads to ...

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3772 ---

Re: [VOTE] 0.11.0.1 RC0

2017-09-06 Thread Damian Guy
Resending as i wasn't part of the kafka-clients mailing list On Tue, 5 Sep 2017 at 21:34 Damian Guy wrote: > Hello Kafka users, developers and client-developers, > > This is the first candidate for release of Apache Kafka 0.11.0.1. > > This is a bug fix release and it includes fixes and improvem

[GitHub] kafka pull request #3776: KAFKA-5819: Add Joined class and relevant KStream ...

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3776 ---

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5817; Add Serialized class and overloads to KStream#groupBy and -- [...truncated 2.02 MB...] org.apache.kafka.clients.producer.KafkaProduc

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5819; Add Joined class and relevant KStream join overloads -- [...truncated 921.66 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > s

[GitHub] kafka pull request #3801: MINOR: Include response in request log

2017-09-06 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3801 MINOR: Include response in request log It's implemented such that there is no overhead if request logging is disabled. Also: - Reduce metrics computation duplication in `updateRequestM

[jira] [Created] (KAFKA-5844) Add groupBy(KeyValueMapper, Serialized) to KTable

2017-09-06 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5844: - Summary: Add groupBy(KeyValueMapper, Serialized) to KTable Key: KAFKA-5844 URL: https://issues.apache.org/jira/browse/KAFKA-5844 Project: Kafka Issue Type: Sub-tas

[GitHub] kafka pull request #3802: KAFKA-5844: add groupBy(selector, serialized) to K...

2017-09-06 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3802 KAFKA-5844: add groupBy(selector, serialized) to Ktable add `KTable#groupBy(KeyValueMapper, Serialized)` and deprecate the overload with `Serde` params You can merge this pull request into a Git repo

Re: [VOTE] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-09-06 Thread Tom Bentley
Unfortunately I've had to make a small change to the ElectPreferredLeadersResult, because exposing a Map> was incompatible with the case where electPreferredLeaders() was called with a null partitions argument. The change exposes methods to access the map which return futures, rather than exposing

Re: Fw: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Ted Yu
Looks good to me. bq. *specifying the --zookeeper for the farmer * *farmer -> former* On Wed, Sep 6, 2017 at 1:42 AM, Paolo Patierno wrote: > Hi devs, > > > I haven't seen any votes for this since last month. > > Is there something that should be addressed in the KIP (it didn't have any > comm

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Ismael Juma
Thanks for the KIP. +1 (binding). Please make it clear in the KIP that removal will happen in 2.0.0. Ismael On Tue, Aug 8, 2017 at 11:53 AM, Paolo Patierno wrote: > Hi devs, > > > I didn't see any more comments about this KIP. The JIRAs related to the > first step (so making --new-consumer as d

Problem: undesired data in a temporary topic

2017-09-06 Thread Louis Verret
Hello, I am writing to you because we are facing a technical issue regarding the development of our data streaming application using Kafka. Our application consists of two Java main: The Producer and The Consumer . The Producer periodically reads data lines from a file and sends them (as a m

[jira] [Created] (KAFKA-5845) KafkaController should send LeaderAndIsrRequest to brokers which starts very soon after shutdown

2017-09-06 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5845: --- Summary: KafkaController should send LeaderAndIsrRequest to brokers which starts very soon after shutdown Key: KAFKA-5845 URL: https://issues.apache.org/jira/browse/KAFKA-5845

[GitHub] kafka pull request #3803: KAFKA-5845; KafkaController should send LeaderAndI...

2017-09-06 Thread lindong28
GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/3803 KAFKA-5845; KafkaController should send LeaderAndIsrRequest to broker which starts very soon after shutdown You can merge this pull request into a Git repository by running: $ git pull http

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-06 Thread Sumant Tambe
120 seconds default sounds good to me. Throwing ConfigException instead of WARN is fine. Added clarification that the producer waits the full request.timeout.ms for the in-flight request. This implies that user might be notified of batch expiry while a batch is still in-flight. I don't recall if w

[GitHub] kafka pull request #3702: KAFKA-5756 Synchronization issue on flush

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3702 ---

[jira] [Resolved] (KAFKA-5756) Synchronization issue on flush

2017-09-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5756?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-5756. Resolution: Fixed Fix Version/s: 0.11.0.1 1.0.0 Issue resolved by

[GitHub] kafka-site pull request #75: Fix HTML markup

2017-09-06 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka-site/pull/75 Fix HTML markup You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka-site hotfix Alternatively you can review and apply these changes a

[GitHub] kafka-site issue #75: Fix HTML markup

2017-09-06 Thread mjsax
Github user mjsax commented on the issue: https://github.com/apache/kafka-site/pull/75 Call for review and merging @guozhangwang ---

[GitHub] kafka pull request #3669: KAFKA-5726: KafkaConsumer.subscribe() overload tha...

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3669 ---

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5756; Connect WorkerSourceTask synchronization issue on flush -- [...truncated 922.40 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > sho

[GitHub] kafka pull request #3804: MINOR: fixed typos

2017-09-06 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3804 MINOR: fixed typos You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka hotfix Alternatively you can review and apply these changes as the

Re: integration between pull request and JIRA

2017-09-06 Thread Matthias J. Sax
You can subscribe to single PR if you want, too. (That actually happens, when you get tagged or comment on one, ie, you get auto subscribed to the PR.) There is a "Subscribe" button on the right hand side. -Matthias On 9/5/17 8:57 PM, Ted Yu wrote: > bq. I did get tagged or I did comment on etc

[GitHub] kafka pull request #3805: MINOR: Implement toString for NetworkClient

2017-09-06 Thread bbaugher
GitHub user bbaugher opened a pull request: https://github.com/apache/kafka/pull/3805 MINOR: Implement toString for NetworkClient You can merge this pull request into a Git repository by running: $ git pull https://github.com/bbaugher/kafka inFlightRequest_toString Alternativ

[GitHub] kafka-site pull request #75: Fix HTML markup

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/75 ---

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5726; KafkaConsumer.subscribe() overload that takes just Pattern -- [...truncated 921.84 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[GitHub] kafka-site pull request #74: Added portoseguro, micronauticsresearch & cj lo...

2017-09-06 Thread ewencp
Github user ewencp commented on a diff in the pull request: https://github.com/apache/kafka-site/pull/74#discussion_r137371720 --- Diff: powered-by.html --- @@ -412,7 +412,22 @@ "logo": "rabobank.jpg", "logoBgColor": "#ff", "des

Build failed in Jenkins: kafka-0.11.0-jdk7 #301

2017-09-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5756; Connect WorkerSourceTask synchronization issue on flush -- [...truncated 2.46 MB...] org.apache.kafka.streams.state.internals.CompositeRe

[GitHub] kafka-site issue #50: Typo in kafka 0.10.2 topics operation doc

2017-09-06 Thread ewencp
Github user ewencp commented on the issue: https://github.com/apache/kafka-site/pull/50 @jlisam Could you close out this PR since the changes were retargeted to the main Kafka repo? ---

[GitHub] kafka-site issue #50: Typo in kafka 0.10.2 topics operation doc

2017-09-06 Thread jlisam
Github user jlisam commented on the issue: https://github.com/apache/kafka-site/pull/50 sure thing ---

[GitHub] kafka-site pull request #50: Typo in kafka 0.10.2 topics operation doc

2017-09-06 Thread jlisam
Github user jlisam closed the pull request at: https://github.com/apache/kafka-site/pull/50 ---

[jira] [Created] (KAFKA-5846) Use singleton NoOpConsumerRebalanceListener in subscribe() call where listener is not specified

2017-09-06 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-5846: - Summary: Use singleton NoOpConsumerRebalanceListener in subscribe() call where listener is not specified Key: KAFKA-5846 URL: https://issues.apache.org/jira/browse/KAFKA-5846 Proje

[jira] [Created] (KAFKA-5847) Plugin option to filter consumer and producer messages on the broker

2017-09-06 Thread Brian Hawkins (JIRA)
Brian Hawkins created KAFKA-5847: Summary: Plugin option to filter consumer and producer messages on the broker Key: KAFKA-5847 URL: https://issues.apache.org/jira/browse/KAFKA-5847 Project: Kafka

Re: [VOTE] KIP-175: Additional '--describe' views for ConsumerGroupCommand

2017-09-06 Thread Guozhang Wang
Sorry for the delay, just made a pass over the wiki page. +1 On Mon, Aug 14, 2017 at 12:02 AM, Manikumar wrote: > +1 (non-binding) > > On Fri, Aug 11, 2017 at 8:09 PM, Mickael Maison > wrote: > > > +1 non-binding, thanks Vahid > > > > On Wed, Aug 9, 2017 at 9:31 PM, Jason Gustafson > > wrote:

1.0.0 KIPs Update

2017-09-06 Thread Guozhang Wang
Hello folks, This is a heads up on 1.0.0 progress: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=71764913 We have one week left towards the KIP deadline, which is Sept. 13th. There are still a lot of KIPs that under discussion / voting process. For the KIP proposer, please kee

Re: [VOTE] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-09-06 Thread Guozhang Wang
Thanks Attila, Please note that the feature freeze deadline is about two weeks away to have your PR to be merged to trunk for the coming 1.0.0 release Guozhang On Wed, Sep 6, 2017 at 1:31 AM, Attila Kreiner wrote: > Hi All, > > No more votes received for this, so the final decision is: accep

Re: 1.0.0 KIPs Update

2017-09-06 Thread Vahid S Hashemian
Hi Guozhang, Thanks for the heads-up. Can KIP-163 be added to the list? The proposal for this KIP is accepted, and the PR is ready for review. Thanks. --Vahid From: Guozhang Wang To: "dev@kafka.apache.org" Date: 09/06/2017 01:45 PM Subject:1.0.0 KIPs Update Hello folks,

Re: 1.0.0 KIPs Update

2017-09-06 Thread Guozhang Wang
Hi Vahid, Yes I have just added it while sending this email :) Guozhang On Wed, Sep 6, 2017 at 1:54 PM, Vahid S Hashemian wrote: > Hi Guozhang, > > Thanks for the heads-up. > > Can KIP-163 be added to the list? > The proposal for this KIP is accepted, and the PR is ready for review. > > Thank

Re: [VOTE] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-09-06 Thread Ismael Juma
The PR for this was merged an hour or two ago. Ismael On 6 Sep 2017 9:51 pm, "Guozhang Wang" wrote: > Thanks Attila, > > Please note that the feature freeze deadline is about two weeks away to > have your PR to be merged to trunk for the coming 1.0.0 release > > > Guozhang > > > On Wed, Sep 6,

Re: [VOTE] KIP-189 - Improve principal builder interface and add support for SASL

2017-09-06 Thread Jason Gustafson
Hi All, When implementing this, I found that the SecurityProtocol class has some internal details which we might not want to expose to users (in particular to enable testing). Since it's still useful to know the security protocol in use in some cases, and since the security protocol names are alre

[jira] [Created] (KAFKA-5848) KafkaConsumer should validate topics/TopicPartitions on subscribe/assign

2017-09-06 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5848: -- Summary: KafkaConsumer should validate topics/TopicPartitions on subscribe/assign Key: KAFKA-5848 URL: https://issues.apache.org/jira/browse/KAFKA-5848 Project: K

Re: [VOTE] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-09-06 Thread Guozhang Wang
Thanks for confirming. On Wed, Sep 6, 2017 at 2:07 PM, Ismael Juma wrote: > The PR for this was merged an hour or two ago. > > Ismael > > On 6 Sep 2017 9:51 pm, "Guozhang Wang" wrote: > > > Thanks Attila, > > > > Please note that the feature freeze deadline is about two weeks away to > > have y

Re: [VOTE] KIP-188 - Add new metrics to support health checks

2017-09-06 Thread Jun Rao
Hi, Raijini, Thanks for the KIP. +1. Just a minor comment. Since we only measure MessageConversionsTimeMs at the request type level, is it useful to collect the following metrics at the topic level? *MBean*: kafka.server:type=BrokerTopicMetrics,name=FetchMessageConversionsPerSec,topic=([-.\w]+)

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-06 Thread Jun Rao
Hi, Sumant, The diagram in the wiki seems to imply that delivery.timeout.ms doesn't include the batching time. For retries, probably we can just default it to MAX_INT? Thanks, Jun On Wed, Sep 6, 2017 at 10:26 AM, Sumant Tambe wrote: > 120 seconds default sounds good to me. Throwing ConfigEx

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5726; KafkaConsumer.subscribe() overload that takes just Pattern -- [...truncated 4.85 MB...] org.apache.kafka.streams.integration.KStreamsFineG

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Guozhang Wang
+1. Thanks. On Wed, Sep 6, 2017 at 7:57 AM, Ismael Juma wrote: > Thanks for the KIP. +1 (binding). Please make it clear in the KIP that > removal will happen in 2.0.0. > > Ismael > > On Tue, Aug 8, 2017 at 11:53 AM, Paolo Patierno > wrote: > > > Hi devs, > > > > > > I didn't see any more commen

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-09-06 Thread Colin McCabe
On Wed, Sep 6, 2017, at 00:20, Tom Bentley wrote: > Hi Ted and Colin, > > Thanks for the comments. > > It seems you're both happier with reassign rather than assign, so I'm > happy > to stick with that. > > > On 5 September 2017 at 18:46, Colin McCabe wrote: > > > ... > > > > Do we expect t

[GitHub] kafka-site pull request #74: Added portoseguro, micronauticsresearch & cj lo...

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/74 ---

[GitHub] kafka pull request #3804: MINOR: fixed typos

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3804 ---

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-09-06 Thread Colin McCabe
On Wed, Sep 6, 2017, at 01:18, Tom Bentley wrote: > Hi Colin, > > Thanks for taking the time to respond. > > On 5 September 2017 at 22:22, Colin McCabe wrote: > > > ... > > Why does there need to be a map at all in the API? > > > From a purely technical PoV there doesn't, but doing something

[jira] [Created] (KAFKA-5849) Add partitioned produce consume test

2017-09-06 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5849: -- Summary: Add partitioned produce consume test Key: KAFKA-5849 URL: https://issues.apache.org/jira/browse/KAFKA-5849 Project: Kafka Issue Type: Bug

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: fix typos on web doc's upgrade guide -- [...truncated 921.67 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotResetEpochHistor

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Vahid S Hashemian
+1. Thanks for the KIP. --Vahid From: Guozhang Wang To: "dev@kafka.apache.org" Date: 09/06/2017 03:41 PM Subject:Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools +1. Thanks. On Wed, Sep 6, 2017 at 7:57 AM, Ismael Juma wrote: > Thanks for the KIP. +1

Re: [DISCUSS] KIP-192 - Provide cleaner semantics when idempotence is enabled

2017-09-06 Thread Apurva Mehta
Hi Jason, Thanks for the comments: 1. I have also updated the KIP to indicate that the DuplicateSequenceException could be the new error code returned in the producer callback as a result of tightening up the semantics of the OutOfOrderSequenceException. 2. I think returning the me

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

2017-09-06 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3769: MINOR: Log encountered exception during rebalance

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3769 ---

[GitHub] kafka pull request #3796: MINOR: KIP-138 renaming of string names

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3796 ---

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Log encountered exception during rebalance -- [...truncated 2.03 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest > testClientMod

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

2017-09-06 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3667: KAFKA-5606: Review consumer's RequestFuture usage ...

2017-09-06 Thread jedichien
Github user jedichien closed the pull request at: https://github.com/apache/kafka/pull/3667 ---

A quick question on StickyAssignor

2017-09-06 Thread Hu Xi
Hello Dev, I am a little confused about the statement below in KIP-54 (StickyAssignor things): * if a consumer A has 2+ fewer topic partitions assigned to it compared to another consumer B, none of the topic partitions assigned to B can be assigned to A. Is it a typo or do I misunderstand

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-06 Thread Becket Qin
Hey Sumant, I agree with Jun that we can set the default value of retries to MAX_INT. Initially I was also thinking that retries can be deprecated. But after a second thought, I feel it may not be necessary to deprecate retries. With the newly added delivery.timeout.ms, the producer will expire a

Re: A quick question on StickyAssignor

2017-09-06 Thread Vahid S Hashemian
Hi Hu Xi, This is a typo. It should read if a consumer A has 2+ fewer topic partitions assigned to it compared to another consumer B, none of the topic partitions assigned to A can be assigned to B. For this assignor, the partition movement should not widen the existing balance gap among cons

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

2017-09-06 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3714: close iterator on doc example

2017-09-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3714 ---

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-06 Thread Sumant Tambe
I'm not sure whether it's a good idea to have two different ways to control expiration. One option as you suggested is to expire batches based on whichever happens first (exceed delivery.timeout.ms or exhaust retries). Second option is to effectively ignore retries even if it's a very low value. Ju

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

2017-09-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: close iterator on doc example -- [...truncated 921.81 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotResetEpochHistoryTailI

答复: A quick question on StickyAssignor

2017-09-06 Thread Hu Xi
Thanks Hashemian. It's clear to me now 😊 发件人: Vahid S Hashemian 发送时间: 2017年9月7日 12:41 收件人: dev@kafka.apache.org 主题: Re: A quick question on StickyAssignor Hi Hu Xi, This is a typo. It should read if a consumer A has 2+ fewer topic partitions assigned to it comp

[GitHub] kafka pull request #3806: KAFKA-5797: Handle metadata not available in store...

2017-09-06 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3806 KAFKA-5797: Handle metadata not available in store registration This is the backport of #3748 for trunk You can merge this pull request into a Git repository by running: $ git pull https:/

[jira] [Created] (KAFKA-5850) Py4JJavaError: An error occurred while calling o40.loadClass. : java.lang.ClassNotFoundException: org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper

2017-09-06 Thread Saurabh Bidwai (JIRA)
Saurabh Bidwai created KAFKA-5850: - Summary: Py4JJavaError: An error occurred while calling o40.loadClass. : java.lang.ClassNotFoundException: org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper Key: KAFKA-5850