[jira] [Commented] (KAFKA-1101) Need better java api for embedding kafkaserver in a java container app

2017-02-06 Thread Enrico Olivelli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853657#comment-15853657 ] Enrico Olivelli commented on KAFKA-1101: I run embedded Kafka too, the fact the it

[jira] [Commented] (KAFKA-1101) Need better java api for embedding kafkaserver in a java container app

2017-02-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853740#comment-15853740 ] Ismael Juma commented on KAFKA-1101: In the meantime, you can configure a SecurityMana

[jira] [Assigned] (KAFKA-4513) Support migration of old consumers to new consumers without downtime

2017-02-06 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4513?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman reassigned KAFKA-4513: --- Assignee: Onur Karaman > Support migration of old consumers to new consumers without downtime

[jira] [Commented] (KAFKA-1101) Need better java api for embedding kafkaserver in a java container app

2017-02-06 Thread Enrico Olivelli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853957#comment-15853957 ] Enrico Olivelli commented on KAFKA-1101: Awesome solution! it works, here is my co

[jira] [Commented] (KAFKA-4715) Consumer/Producer config does not work with related enums

2017-02-06 Thread Mathias Kub (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853975#comment-15853975 ] Mathias Kub commented on KAFKA-4715: Hi, I've introduced a small changeset to include

[jira] [Comment Edited] (KAFKA-4715) Consumer/Producer config does not work with related enums

2017-02-06 Thread Mathias Kub (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853975#comment-15853975 ] Mathias Kub edited comment on KAFKA-4715 at 2/6/17 1:23 PM: Hi

[jira] [Commented] (KAFKA-1101) Need better java api for embedding kafkaserver in a java container app

2017-02-06 Thread Enrico Olivelli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15853984#comment-15853984 ] Enrico Olivelli commented on KAFKA-1101: Just for completeness, I had to add some

[jira] [Commented] (KAFKA-4703) Test with two SASL_SSL listeners with different JAAS contexts

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4703?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854155#comment-15854155 ] ASF GitHub Bot commented on KAFKA-4703: --- GitHub user baluchicken opened a pull reque

[GitHub] kafka pull request #2506: KAFKA-4703 Test with two SASL_SSL listeners with d...

2017-02-06 Thread baluchicken
GitHub user baluchicken opened a pull request: https://github.com/apache/kafka/pull/2506 KAFKA-4703 Test with two SASL_SSL listeners with different JAAS contexts @ijuma plz review You can merge this pull request into a Git repository by running: $ git pull https://github.com/ba

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Eno Thereska
Hi Damian, I am trying to figure out if this handles a common enough failure scenario. It seems to me this handles transient failures: a server with an instance fails, then comes back up shortly and the instance recovers quickly by reading the checkpoint file. Permanent failures, where the ser

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Damian Guy
Hi Eno, The state is on local disk, so having the checkpoint in a topic won't help. If the host fails permanently, then all of the local state is gone. Starting on another host requires restoring from the earliest offset. Thanks, Damian On Mon, 6 Feb 2017 at 14:58 Eno Thereska wrote: > Hi Dami

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Eno Thereska
Makes sense, thanks. Eno > On 6 Feb 2017, at 15:01, Damian Guy wrote: > > Hi Eno, > > The state is on local disk, so having the checkpoint in a topic won't help. > If the host fails permanently, then all of the local state is gone. > Starting on another host requires restoring from the earliest

[jira] [Commented] (KAFKA-4715) Consumer/Producer config does not work with related enums

2017-02-06 Thread Bryan Baugher (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854193#comment-15854193 ] Bryan Baugher commented on KAFKA-4715: -- If thats the plan we can just use the existin

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Mathieu Fenniak
Hi Matthias, I use a few of the methods that you're pointing out that will be deprecated and don't have an apparent alternative, so I wanted to just let you know what they are and what my use-cases are for them. First of all, I use a combination of DSL and PAPI in the same application very happil

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Tom Crayford
I think the updated wiki page makes sense with respect to ACLs, there seems to be little potential for abuse there (other than the noted and known issues). I am going to note that this is a major complexity increase for Kafka, and that I'm concerned about performance impact (the JVM is quite… peda

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Damian Guy
Hi Matthias, Thanks for the KIP. Should TopologyBuilder#build() and KStreamBuilder#build accept a StreamsConfig as an argument? Should we add a KStreamBuilder#topologyBuilder() for cases where people want to mix and match DSL & PAPI? Or do you think we already provide enough support for that via

[GitHub] kafka-site issue #45: Manual edits needed for 0.10.2 release

2017-02-06 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/45 @ewencp @derrickdoo for reviews. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature ena

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854436#comment-15854436 ] Matthias J. Sax commented on KAFKA-4564: With new client compatibility in {{0.10.2

[jira] [Commented] (KAFKA-4264) kafka-server-stop.sh fails is Kafka launched via kafka-server-start.sh

2017-02-06 Thread Jeff Maxwell (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854484#comment-15854484 ] Jeff Maxwell commented on KAFKA-4264: - The pull request [1984|https://github.com/apach

[GitHub] kafka pull request #2273: KAFKA-4555: Using Hamcrest for expressive intent i...

2017-02-06 Thread rekhajoshm
Github user rekhajoshm closed the pull request at: https://github.com/apache/kafka/pull/2273 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Commented] (KAFKA-4555) Using Hamcrest for easy intent expression in tests

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4555?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854500#comment-15854500 ] ASF GitHub Bot commented on KAFKA-4555: --- Github user rekhajoshm closed the pull requ

[GitHub] kafka pull request #2486: KAFKA-4724: Clean up of state directories can poss...

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2486 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Resolved] (KAFKA-4724) Clean up of state directories can possibly remove stores that are about to be used by another thread

2017-02-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4724. -- Resolution: Fixed Issue resolved by pull request 2486 [https://github.com/apache/kafka/pull/2486

[jira] [Commented] (KAFKA-4724) Clean up of state directories can possibly remove stores that are about to be used by another thread

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854554#comment-15854554 ] ASF GitHub Bot commented on KAFKA-4724: --- Github user asfgit closed the pull request

Request Wiki edit permission for stevenschlansker

2017-02-06 Thread Steven Schlansker
Hi dev@, I recently signed up for the Kafka Wiki as 'stevenschlansker' and would like to contribute a KIP. May I please have edit access to do so? Thanks! signature.asc Description: Message signed with OpenPGP using GPGMail

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote: > Thanks for the proposal Colin. A few comments below: Thanks for taking a look, Guozhang. > > 1. There are a couple of classes that looks new to me but not defined > anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is > th

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote: > Thanks for the proposal Colin. A few comments below: > > 1. There are a couple of classes that looks new to me but not defined > anywhere. For example: NewTopic (topic name and configs?), TopicInfo (is > this a wrapper of MetadataResponse.Topic

[jira] [Assigned] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2017-02-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-4461: Assignee: Adrian McCague > When using ProcessorTopologyTestDriver, the combination of map a

[GitHub] kafka pull request #2499: KAFKA-4461 Added support to ProcessorTopologyTestD...

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2499 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854608#comment-15854608 ] ASF GitHub Bot commented on KAFKA-4461: --- Github user asfgit closed the pull request

[jira] [Resolved] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2017-02-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4461. -- Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2499 [https://gi

[jira] [Commented] (KAFKA-4461) When using ProcessorTopologyTestDriver, the combination of map and .groupByKey does not produce any result

2017-02-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854609#comment-15854609 ] Guozhang Wang commented on KAFKA-4461: -- [~amccague] Thanks for the patch! I have adde

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Colin McCabe
On Fri, Feb 3, 2017, at 16:57, Dong Lin wrote: > Thanks for the reply, Colin. I have some comments inline. Hi Dong L., > > In addition, I also have some comments regarding the Future() in response > to your latest email. As Ismael mentioned, we have added > purgeDataBefore() > API in AdminClient

[GitHub] kafka pull request #2491: [KAFKA-4728] KafkaConsumer#commitSync should clone...

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2491 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4728) KafkaConsumer#commitSync should clone its input

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4728?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854620#comment-15854620 ] ASF GitHub Bot commented on KAFKA-4728: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-4728) KafkaConsumer#commitSync should clone its input

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4728: --- Resolution: Fixed Fix Version/s: 0.10.2.0 Status: Resolved (was: Patch Avail

Re: Request Wiki edit permission for stevenschlansker

2017-02-06 Thread Guozhang Wang
Done. Guozhang On Mon, Feb 6, 2017 at 10:02 AM, Steven Schlansker < stevenschlans...@gmail.com> wrote: > Hi dev@, > > I recently signed up for the Kafka Wiki as 'stevenschlansker' and would > like to contribute a KIP. > May I please have edit access to do so? Thanks! > > -- -- Guozhang

[jira] [Created] (KAFKA-4738) Remove generic type of class ClientState

2017-02-06 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4738: -- Summary: Remove generic type of class ClientState Key: KAFKA-4738 URL: https://issues.apache.org/jira/browse/KAFKA-4738 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-4728) KafkaConsumer#commitSync should clone its input

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4728: --- Fix Version/s: (was: 0.10.2.0) 0.10.3.0 > KafkaConsumer#commitSync shou

[GitHub] kafka pull request #2325: KAFKA-4441 Monitoring incorrect during topic creat...

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2325 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4441) Kafka Monitoring is incorrect during rapid topic creation and deletion

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854679#comment-15854679 ] ASF GitHub Bot commented on KAFKA-4441: --- Github user asfgit closed the pull request

[jira] [Resolved] (KAFKA-4441) Kafka Monitoring is incorrect during rapid topic creation and deletion

2017-02-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-4441. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2325 [https://github.com/apa

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-06 Thread Michael Noll
> Starting on another host requires restoring from the earliest offset. Btw, there's a special scenario where a full restore is not required: When the local storage (volume) is being re-used, e.g. when a container uses a storage mount that will be re-used by a new container in case the original o

[GitHub] kafka pull request #2507: HOTFIX: Do Not use unlimited num messages

2017-02-06 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2507 HOTFIX: Do Not use unlimited num messages Removed readKeyValues() that give UNLIMITED_MESSAGES which will doom to exhaust all wait time, as all its callers actually do provide the expected num

[jira] [Created] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
Vipul Singh created KAFKA-4739: -- Summary: KafkaConsumer poll going into an infinite loop Key: KAFKA-4739 URL: https://issues.apache.org/jira/browse/KAFKA-4739 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854712#comment-15854712 ] Jason Gustafson commented on KAFKA-2857: [~vahid] Sounds reasonable. At least we s

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

[jira] [Created] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread JIRA
Sébastien Launay created KAFKA-4740: --- Summary: Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop Key: KAFKA-4740 URL: https://issues.apache.org/jira/browse/K

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Jeff Widman
For individual consumer groups, it would be nice if the admin client made it possible to fetch consumer offsets for the entire consumer group. Then we don't have to manually assemble this outside of the admin client interface. On Feb 6, 2017 11:41 AM, "Colin McCabe" wrote: > On Fri, Feb 3, 2017,

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854731#comment-15854731 ] Jason Gustafson commented on KAFKA-4739: [~neoeahit] Thanks for the report. Would

[jira] [Comment Edited] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854731#comment-15854731 ] Jason Gustafson edited comment on KAFKA-4739 at 2/6/17 9:09 PM:

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854780#comment-15854780 ] Jason Gustafson commented on KAFKA-4740: Thanks for the report. The intention of t

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
Mathieu, Damian, thanks a lot for your feedback. It's very valuable to see what, how and why people are using certain methods right now. We don't want to lock people out (that's why we put this KIP on users list, too) and we want to keep the ability to mix-and-match DSL and Processor API. Furthe

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854791#comment-15854791 ] Jason Gustafson commented on KAFKA-4740: That said, one thing I realized looking a

[GitHub] kafka pull request #2508: MINOR: Fix import for streams broker compatibility...

2017-02-06 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/2508 MINOR: Fix import for streams broker compatibility test to use new DEV_BRANCH constant You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/ka

[GitHub] kafka pull request #2508: MINOR: Fix import for streams broker compatibility...

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2508 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Guozhang Wang
Some follow-up on 2) / 3) below. On Mon, Feb 6, 2017 at 11:21 AM, Colin McCabe wrote: > On Fri, Feb 3, 2017, at 16:25, Guozhang Wang wrote: > > Thanks for the proposal Colin. A few comments below: > > Thanks for taking a look, Guozhang. > > > > > 1. There are a couple of classes that looks new t

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread Jianbin Wei
In the specify group information, can we also return information like partition assignment for each member, the lag/offset of each member/partition? It would be useful for Ops/Admin regarding the health of the consumer group. Regards, -- Jianbin > On Feb 6, 2017, at 13:54, Guozhang Wang wrot

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854842#comment-15854842 ] Vipul Singh commented on KAFKA-4739: [~hachikuji], I have updated the description of t

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

Build failed in Jenkins: kafka-0.10.2-jdk7 #66

2017-02-06 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4441; Monitoring incorrect during topic creation and deletion -- [...truncated 4100 lines...] kafka.api.ProducerFailureHandlingTest > testNotEnoughReplicas STARTED k

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854847#comment-15854847 ] Sébastien Launay commented on KAFKA-4740: - bq. it seems a little riskier since app

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854894#comment-15854894 ] Jason Gustafson commented on KAFKA-4739: bq. An observation from our side: it look

[GitHub] kafka pull request #2507: HOTFIX: Do Not use unlimited num messages

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2507 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Apurva Mehta
Hi Tom, I updated the KIP with a note our plans for performance testing: https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-Performance Thanks for pointing that out. Regards, Apurva

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854910#comment-15854910 ] Sébastien Launay commented on KAFKA-4740: - That sounds like a good trade off betwe

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854905#comment-15854905 ] James Cheng commented on KAFKA-4739: I wonder if we're seeing the same thing as this.

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854923#comment-15854923 ] Jason Gustafson commented on KAFKA-4739: [~wushujames] Sounds a bit different. Not

[jira] [Assigned] (KAFKA-4738) Remove generic type of class ClientState

2017-02-06 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sharad reassigned KAFKA-4738: - Assignee: Sharad > Remove generic type of class ClientState > >

[jira] [Updated] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4739: --- Description: We are seeing an issue with our kafka consumer where it seems to go into an infinite loo

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854943#comment-15854943 ] Vipul Singh commented on KAFKA-4739: consumer config: https://gist.github.com/neoeahi

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Sagar Sadashiv Patwardhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854956#comment-15854956 ] Sagar Sadashiv Patwardhan commented on KAFKA-4739: -- [~hachikuji] I am wit

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854869#comment-15854869 ] Jason Gustafson commented on KAFKA-4740: Yeah, auto-commit is the case I was think

[jira] [Comment Edited] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15854943#comment-15854943 ] Vipul Singh edited comment on KAFKA-4739 at 2/6/17 11:43 PM: -

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

2017-02-06 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2504: KAFKA-4735; Fix deadlock issue during MM shutdown

2017-02-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2504 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855001#comment-15855001 ] Vahid Hashemian commented on KAFKA-2857: [~hachikuji] Great. I assume you are sugg

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Jason Gustafson
Hey Tom, Re; complexity. This is always a tradeoff with new features. The changes we've made during the design and review process have greatly simplified the implementation for clients, and especially for the consumer, but there is nontrivial work needed here to support transactions on the produce

[jira] [Commented] (KAFKA-4735) Fix deadlock issue during MM shutdown

2017-02-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4735?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855007#comment-15855007 ] ASF GitHub Bot commented on KAFKA-4735: --- Github user asfgit closed the pull request

Jenkins build is back to normal : kafka-0.10.2-jdk7 #67

2017-02-06 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855034#comment-15855034 ] Jason Gustafson commented on KAFKA-2857: Hmm, I thought we used the admin client f

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Gwen Shapira
I like the cleanup a lot :) The cleaner lines between PAPI and DSL are very helpful to beginners who try to make sense of a long list of methods. I noticed that the "merge" method is still part of StreamBuilder. I thought it belongs inside KStream. Merge seems a lot like the SQL "union" operator,

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855052#comment-15855052 ] Vahid Hashemian commented on KAFKA-2857: Sorry, you are right. This would become a

[jira] [Updated] (KAFKA-4735) Fix deadlock issue during MM shutdown

2017-02-06 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-4735: Affects Version/s: 0.10.2.0 Fix Version/s: 0.10.3.0 > Fix deadlock issue during MM shutdown

[jira] [Resolved] (KAFKA-4735) Fix deadlock issue during MM shutdown

2017-02-06 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin resolved KAFKA-4735. - Resolution: Fixed > Fix deadlock issue during MM shutdown > -

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855098#comment-15855098 ] Jason Gustafson commented on KAFKA-4739: [~sagar8192] Unfortunately, there is no s

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-02-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855122#comment-15855122 ] Jason Gustafson commented on KAFKA-4740: That's a good point. We'd have to first r

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
Gwen, thanks for your feedback. I completely agree that KStreamBuilder#merge() is miss placed and should belong to KStream. However, I wanted to keep this KIP focus on one thing. As mentioned in a previous reply, we plan to have at least one more KIP to clean up DSL -- this future KIP should inc

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-06 Thread Matthias J. Sax
Steven, Thanks for your KIP. I move this discussion to dev mailing list -- KIPs need to be discussed there (and can be cc'ed to user list). Can you also add the KIP to the table "KIPs under discussion": https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovemen

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-06 Thread Apurva Mehta
Hello, I have gone ahead and updated the KIP wiki with a summary of the changes to the RPC protocol. The KIP wiki should now have _all_ the public facing changes being proposed. The proposed changes were always in the Google doc, and now we are simply making good on our promise to copy them over

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-06 Thread Jun Rao
Hi, Dong, Thanks for the proposal. A few quick questions/comments. 1. Do you know why your stress test loses 15% of the throughput with the one-broker-per-disk setup? 2. In the KIP, it wasn't super clear to me what /broker/topics/[topic]/partitions/[partitionId]/controller_managed_state represen

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Gwen Shapira
Sounds good :) On Mon, Feb 6, 2017 at 5:40 PM, Matthias J. Sax wrote: > Gwen, > > thanks for your feedback. > > I completely agree that KStreamBuilder#merge() is miss placed and should > belong to KStream. However, I wanted to keep this KIP focus on one thing. > > As mentioned in a previous reply

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-06 Thread radai
even assuming all consumers use kafka for offset storage, would it be possible to get this information from a single broker without "reaching out" to all brokers in a cluster? On Mon, Feb 6, 2017 at 2:05 PM, Jianbin Wei wrote: > In the specify group information, can we also return information li

[jira] [Commented] (KAFKA-4731) Add event-based session windows

2017-02-06 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855204#comment-15855204 ] Bill Bejeck commented on KAFKA-4731: [~mjsax] Is this Jira pending any further API red

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-06 Thread Gwen Shapira
I've read the wiki and am unclear about the proposal. Can you provide something like a Javadoc for peek()? What would this method do? Also, forgive me if I'm missing an important point here, but can't I put the println statement in a map()? On Mon, Feb 6, 2017 at 5:48 PM, Matthias J. Sax wrote:

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-06 Thread Dong Lin
Hey Jun, Thanks for the review! Please see reply inline. On Mon, Feb 6, 2017 at 6:21 PM, Jun Rao wrote: > Hi, Dong, > > Thanks for the proposal. A few quick questions/comments. > > 1. Do you know why your stress test loses 15% of the throughput with the > one-broker-per-disk setup? > I think i

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-06 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855232#comment-15855232 ] huxi commented on KAFKA-4739: - Seems that FETCH requests time out every 40 seconds and 40 seco

  1   2   >