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

2016-12-13 Thread Sriram Subramanian
I am not sure if it is a good idea to support both init() and lazy initialization. The ideal state would have been to implement init as a non blocking api and have the rest of the methods throw uninitialized exception if init was not called. This would ensure that init can still be used by other no

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-13 Thread Ismael Juma
Hi Xavier, Thanks for the KIP. If Java had declaration site variance (proposed for a future Java version[1]), we'd mark function parameters as contravariant (i.e. "super") and the result as covariant (i.e. "extends"). In the meantime, we have to use the wildcards at use site as per your proposal.

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

2016-12-13 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4516: When a CachingStateStore is closed it should clear its -- [...truncated 26860 lines...] org.apache.kafka.streams.StreamsConfigTest > shouldBeSupportNonPrefi

[jira] [Commented] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-13 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744529#comment-15744529 ] Jiangjie Qin commented on KAFKA-4529: - Sure, I'll submit a patch. > tombstone may be

[jira] [Assigned] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-13 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin reassigned KAFKA-4529: --- Assignee: Jiangjie Qin > tombstone may be removed earlier than it should > --

[jira] [Work started] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-13 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4529 started by Jiangjie Qin. --- > tombstone may be removed earlier than it should > ---

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

2016-12-13 Thread Jay Kreps
Hey Ismael, Yes, you are correct, I remember now why didn't do that. I rescind that suggestion. I still think lazy initialization is more in keeping with what we've done if feasible. -Jay On Mon, Dec 12, 2016 at 11:36 PM, Ismael Juma wrote: > Hi Jay, > > I like the idea of having a single `ini

Re: [VOTE] KIP-84: Support SASL SCRAM mechanisms

2016-12-13 Thread Rajini Sivaram
Jun, Any thoughts on reducing the number of mechanisms and supporting only SCRAM-SHA-256 and SCRAM-SHA-512? Thank you, Rajini On Fri, Dec 2, 2016 at 2:44 PM, Ismael Juma wrote: > Thanks Rajini. Let's see what Jun says about limiting the number of SHA > variants. Either way, +1 from me. > > Is

[jira] [Created] (KAFKA-4530) cant stop kafka server

2016-12-13 Thread xin (JIRA)
xin created KAFKA-4530: -- Summary: cant stop kafka server Key: KAFKA-4530 URL: https://issues.apache.org/jira/browse/KAFKA-4530 Project: Kafka Issue Type: Bug Components: admin Affects Vers

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-13 Thread Jay Kreps
Hey Ismael, Yeah I think we are both saying the same thing---removing only works if you have a truly optimal strategy. Actually even dynamically computing a reasonable default isn't totally obvious (do you set fetch.max.bytes to equal buffer.memory to try to queue up as much data in the network bu

[jira] [Commented] (KAFKA-4530) cant stop kafka server

2016-12-13 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744578#comment-15744578 ] huxi commented on KAFKA-4530: - Is it a duplicate of [kafka-4297|https://issues.apache.org/jir

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-12-13 Thread Manikumar
Ashish, Thank you for reviewing the KIP. Please see the replies inline. > 1. How to disable delegation token authentication? > > This can be achieved in various ways, however I think reusing delegation > token secret config for this makes sense here. Avoids creating yet another > config and for

[jira] [Commented] (KAFKA-4530) cant stop kafka server

2016-12-13 Thread xin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744587#comment-15744587 ] xin commented on KAFKA-4530: yeah, duplicate > cant stop kafka server >

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-13 Thread Ismael Juma
Makes sense Jay. Mickael, in addition to how we can compute defaults of the other settings from `buffer.memory`, it would be good to specify what is allowed and how we handle the different cases (e.g. what do we do if `max.partition.fetch.bytes` is greater than `buffer.memory`, is that simply not

Improve default Kafka logger settings to prevent extremely high disk space usage issue?

2016-12-13 Thread Jaikiran Pai
We happened to run into a disk space usage issue with Kafka 0.10.0.1 (the version we are using) on one of our production setups this morning. Turns out (log4j) logging from Kafka ended up using 81G and more of disk space. Looking at the files, I see the controller.log itself is 30G and more (fo

[jira] [Commented] (KAFKA-4505) Cannot get topic lag since kafka upgrade from 0.8.1.0 to 0.10.1.0

2016-12-13 Thread Romaric Parmentier (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744686#comment-15744686 ] Romaric Parmentier commented on KAFKA-4505: --- Any idea ? > Cannot get topic lag

Re: Improve default Kafka logger settings to prevent extremely high disk space usage issue?

2016-12-13 Thread Jaikiran Pai
On Tuesday 13 December 2016 03:02 PM, Jaikiran Pai wrote: log4j.logger.kafka.controller=*TRACE,* controllerAppender log4j.additivity.kafka.controller=false log4j.logger.state.change.logger=*TRACE*, stateChangeAppender log4j.additivity.state.change.logger=false Is it intentional to have this

Re: Improve default Kafka logger settings to prevent extremely high disk space usage issue?

2016-12-13 Thread Ismael Juma
Hi Jaikiran, Thanks for raising this. The compatibility issue is that the file name pattern for the rolled files are different. Because there's a simple workaround, it was deemed that it could wait until 0.11.0. This is actually not far off if you consider that we have 4 active KIPs that are propo

[jira] [Commented] (KAFKA-4522) Using Disruptor instead of Array Blocking queue in Kafka Producer

2016-12-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744753#comment-15744753 ] Ismael Juma commented on KAFKA-4522: JCTools is also an option: http://psy-lob-saw.bl

[GitHub] kafka pull request #2249: KAFKA-4473: RecordCollector should handle retriabl...

2016-12-13 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2249 KAFKA-4473: RecordCollector should handle retriable exceptions more strictly The `RecordCollectorImpl` currently drops messages on the floor if an exception is non-null in the producer callback. This w

[jira] [Commented] (KAFKA-4473) RecordCollector should handle retriable exceptions more strictly

2016-12-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744756#comment-15744756 ] ASF GitHub Bot commented on KAFKA-4473: --- GitHub user dguy opened a pull request:

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-13 Thread Michael Pearce
Hi Ewen, Jay, We discussed internally. If this is the reason for not giving a +1 to the below, we assume you want a solution where we can provide all 4 on day one. To do this our idea is to simply leave the compaction policy alone, it continues to work based on null values, but instead we make

[jira] [Updated] (KAFKA-4405) Avoid calling pollNoWakeup unnecessarily

2016-12-13 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4405: Summary: Avoid calling pollNoWakeup unnecessarily (was: Kafka consumer improperly send prefetch req

[jira] [Updated] (KAFKA-4405) Avoid calling pollNoWakeup unnecessarily

2016-12-13 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4405: Description: In KafkaConsumer:poll, the code always calls "pollNoWakeup", which turns out to be exp

[jira] [Commented] (KAFKA-4405) Avoid calling pollNoWakeup unnecessarily

2016-12-13 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744924#comment-15744924 ] Eno Thereska commented on KAFKA-4405: - Done, thanks [~ijuma] > Avoid calling pollNoWa

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-13 Thread Ismael Juma
Yes, this is actually tricky to do in a way where we both have the desired semantics and maintain compatibility. When someone creates a `ProducerRecord` with a `null` value today, the producer doesn't know if it's meant to be a tombstone or not. For V3 messages, it's easy when the constructor that

[DISCUSS] Dormant/Inactive KIPs

2016-12-13 Thread Ismael Juma
Hi all, A while back Grant proposed moving inactive/dormant KIPs to a separate table in the wiki. I think this is a good idea as it will make it easier for people to see the KIPs that are actually active. The list that Grant proposed then was: - KIP-6 - New reassignment partition logic for rebala

[jira] [Created] (KAFKA-4531) Rationalise client configuration validation

2016-12-13 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-4531: Summary: Rationalise client configuration validation Key: KAFKA-4531 URL: https://issues.apache.org/jira/browse/KAFKA-4531 Project: Kafka Issue Type: Improv

Re: [DISCUSS] KIP-83 - Allow multiple SASL PLAIN authenticated Java clients in a single JVM process

2016-12-13 Thread Ismael Juma
Thanks for the KIP. A few comments: 1. The suggestion is to use the JAAS config value as the key to the map in `LoginManager`. The config value can include passwords, so we could potentially end up leaking them if we log the keys of `LoginManager`. This seems a bit dangerous. 2. If someone uses t

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-12-13 Thread Ismael Juma
Hi Rajini, Thanks for the KIP. I think this is useful and users have asked for something like that. I like that you have a scenarios section, do you think you could provide a rough sketch of what a callback handler would look like for the first 2 scenarios? They seem to be the common ones, so it w

[jira] [Commented] (KAFKA-4505) Cannot get topic lag since kafka upgrade from 0.8.1.0 to 0.10.1.0

2016-12-13 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745241#comment-15745241 ] huxi commented on KAFKA-4505: - I am assuming you are still using the old consumer (using 'zook

[jira] [Commented] (KAFKA-4505) Cannot get topic lag since kafka upgrade from 0.8.1.0 to 0.10.1.0

2016-12-13 Thread Romaric Parmentier (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745266#comment-15745266 ] Romaric Parmentier commented on KAFKA-4505: --- Hi Huxi, Yes the application is ru

[jira] [Commented] (KAFKA-4437) Incremental Batch Processing for Kafka Streams

2016-12-13 Thread Avi Flax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745331#comment-15745331 ] Avi Flax commented on KAFKA-4437: - I have a comment but there doesn’t seem to be a mailing

[jira] [Created] (KAFKA-4532) StateStores can be connected to the wrong source topic

2016-12-13 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4532: - Summary: StateStores can be connected to the wrong source topic Key: KAFKA-4532 URL: https://issues.apache.org/jira/browse/KAFKA-4532 Project: Kafka Issue Type: Bu

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-13 Thread Mickael Maison
Thanks for all the feedback. I've updated the KIP with all the details. Below are a few of the main points: - Overall memory usage of the consumer: I made it clear the memory pool is only used to store the raw bytes from the network and that the decompressed/deserialized messages are not stored i

[jira] [Work started] (KAFKA-4532) StateStores can be connected to the wrong source topic

2016-12-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4532 started by Damian Guy. - > StateStores can be connected to the wrong source topic >

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2016-12-13 Thread Rajini Sivaram
Ismael, Thank you for the review. I will add an example. On Tue, Dec 13, 2016 at 1:07 PM, Ismael Juma wrote: > Hi Rajini, > > Thanks for the KIP. I think this is useful and users have asked for > something like that. I like that you have a scenarios section, do you think > you could provide a r

Dev list subscribe

2016-12-13 Thread Rajini Sivaram
Please subscribe me to the Kafka dev list. Thank you, Rajini

[jira] [Created] (KAFKA-4533) subscribe() then poll() on new topic is very slow when subscribed to many topics

2016-12-13 Thread Sergey Alaev (JIRA)
Sergey Alaev created KAFKA-4533: --- Summary: subscribe() then poll() on new topic is very slow when subscribed to many topics Key: KAFKA-4533 URL: https://issues.apache.org/jira/browse/KAFKA-4533 Project:

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-13 Thread Michael Pearce
Hi Ismael Did you see our email this morning, what's your thoughts on this approach to instead we simply have a brand new policy? Cheers Mike Sent using OWA for iPhone From: isma...@gmail.com on behalf of Ismael Juma Sent: Tuesday, December 13, 2016

[jira] [Updated] (KAFKA-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

2016-12-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4532: -- Summary: StateStores can be connected to the wrong source topic resulting in incorrect metadata returned

[jira] [Updated] (KAFKA-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

2016-12-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4532: -- Description: When building a topology with tables and StateStores, the StateStores are mapped to the so

Re: [DISCUSS] KIP-83 - Allow multiple SASL PLAIN authenticated Java clients in a single JVM process

2016-12-13 Thread Edoardo Comar
Thanks for your review, Ismael. First, I am no longer sure KIP-83 is worth keeping as KIP, I created it just before Rajini's https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients With KIP-85 as presented, my proposal has become a simple JIRA, th

[jira] [Comment Edited] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745663#comment-15745663 ] Tom DeVoe edited comment on KAFKA-4477 at 12/13/16 5:20 PM: [~

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745663#comment-15745663 ] Tom DeVoe commented on KAFKA-4477: -- [~junrao] I respectfully disagree, and this is why I

[jira] [Commented] (KAFKA-4437) Incremental Batch Processing for Kafka Streams

2016-12-13 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745672#comment-15745672 ] Matthias J. Sax commented on KAFKA-4437: There is a mailing list thread... Just fo

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-13 Thread Rajini Sivaram
Coordinator starvation: For an implementation based on KIP-72, there will be coordinator starvation without KAFKA-4137 since you would stop reading from sockets when the memory pool is full (the fact that coordinator messages are small doesn't help). I imagine you can work around this by treating c

Re: Dev list subscribe

2016-12-13 Thread Guozhang Wang
Rajini, It's self-service :) https://kafka.apache.org/contact Guozhang On Tue, Dec 13, 2016 at 5:38 AM, Rajini Sivaram wrote: > Please subscribe me to the Kafka dev list. > > > Thank you, > > Rajini > -- -- Guozhang

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745734#comment-15745734 ] Michael Andre Pearce (IG) commented on KAFKA-4477: -- Hi Jun, The stack wa

[GitHub] kafka pull request #2247: MINOR: Fix Streams examples in documentation

2016-12-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2247 --- 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

[GitHub] kafka pull request #2243: KAFKA-4497: LogCleaner appended the wrong offset t...

2016-12-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2243 --- 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-4497) log cleaner breaks on timeindex

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

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745749#comment-15745749 ] Michael Andre Pearce (IG) commented on KAFKA-4477: -- It is worth noting we

[jira] [Resolved] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-4497. Resolution: Fixed [~roschumann], I merged Jiangjie's batch to 0.10.1 branch. Do you think you could give it

Re: Dev list subscribe

2016-12-13 Thread Rajini Sivaram
Sorry, that was a mail sent by mistake. On Tue, Dec 13, 2016 at 5:39 PM, Guozhang Wang wrote: > Rajini, > > It's self-service :) > > https://kafka.apache.org/contact > > Guozhang > > On Tue, Dec 13, 2016 at 5:38 AM, Rajini Sivaram > wrote: > > > Please subscribe me to the Kafka dev list. > > >

[jira] [Comment Edited] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745749#comment-15745749 ] Michael Andre Pearce (IG) edited comment on KAFKA-4477 at 12/13/16 5:49 PM:

[jira] [Updated] (KAFKA-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

2016-12-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4532: -- Description: When building a topology with tables and StateStores, the StateStores are mapped to the so

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745782#comment-15745782 ] Jun Rao commented on KAFKA-4477: [~michael.andre.pearce], the 0.10.1.1 release will need R

[GitHub] kafka pull request #2250: KAFKA-4532: StateStores can be connected to the wr...

2016-12-13 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2250 KAFKA-4532: StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from Interactive Queries When building a topology with tables and StateStores, the StateStore

[jira] [Commented] (KAFKA-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

2016-12-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745809#comment-15745809 ] ASF GitHub Bot commented on KAFKA-4532: --- GitHub user dguy opened a pull request:

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745813#comment-15745813 ] Jun Rao commented on KAFKA-4477: [~tdevoe], thanks for the clarification. Then, it looks s

[GitHub] kafka pull request #2090: KAFKA-4269: Follow up for 0.10.1 branch -update to...

2016-12-13 Thread bbejeck
Github user bbejeck closed the pull request at: https://github.com/apache/kafka/pull/2090 --- 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 ena

[jira] [Commented] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-12-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745818#comment-15745818 ] ASF GitHub Bot commented on KAFKA-4269: --- Github user bbejeck closed the pull request

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745817#comment-15745817 ] Jun Rao commented on KAFKA-4477: [~michael.andre.pearce], [~tdevoe], was the following ex

[DISCUSS] KIP-90 Remove zkClient dependency from Streams

2016-12-13 Thread Hojjat Jafarpour
Hi all, The following is a KIP for removing zkClient dependency from Streams. Please check out the KIP page: https://cwiki.apache.org/confluence/display/KAFKA/KIP-90+-+Remove+zkClient+dependency+from+Streams Thanks, --Hojjat

[jira] [Resolved] (KAFKA-4390) Replace MessageSet usage with client-side equivalents

2016-12-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4390. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2140 [https:

[GitHub] kafka pull request #2140: KAFKA-4390: Replace MessageSet usage with client-s...

2016-12-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2140 --- 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-4390) Replace MessageSet usage with client-side equivalents

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

[jira] [Updated] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom DeVoe updated KAFKA-4477: - Attachment: state_change_controller.tar.gz > Node reduces its ISR to itself, and doesn't recover. Other no

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745919#comment-15745919 ] Tom DeVoe commented on KAFKA-4477: -- Sorry about that [~apurva], I attached the state chan

[GitHub] kafka pull request #2251: KAFKA-4529; Fix the issue that tombstone can be de...

2016-12-13 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2251 KAFKA-4529; Fix the issue that tombstone can be deleted too early. You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka KAFKA-4529

[jira] [Commented] (KAFKA-4529) tombstone may be removed earlier than it should

2016-12-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745956#comment-15745956 ] ASF GitHub Bot commented on KAFKA-4529: --- GitHub user becketqin opened a pull request

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

2016-12-13 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix Streams examples in documentation -- [...truncated 3907 lines...] kafka.utils.CommandLineUtilsTest > testParseEmptyArg STARTED kafka.utils.CommandLineUtilsTe

[jira] [Commented] (KAFKA-4437) Incremental Batch Processing for Kafka Streams

2016-12-13 Thread Avi Flax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745978#comment-15745978 ] Avi Flax commented on KAFKA-4437: - Ah, great, thanks! > Incremental Batch Processing for

[jira] [Created] (KAFKA-4534) StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once.

2016-12-13 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4534: - Summary: StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once. Key: KAFKA-4534 URL: https://issues.apache.org/jira/browse/KAFKA-4534

[jira] [Updated] (KAFKA-4509) Task reusage on rebalance fails for threads on same host

2016-12-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4509?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4509: - Resolution: Fixed Fix Version/s: 0.10.2.0 Status: Resolved (was: Patch Available

[GitHub] kafka pull request #2233: KAFKA-4509: Task reusage on rebalance fails for th...

2016-12-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2233 --- 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-4509) Task reusage on rebalance fails for threads on same host

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

[jira] [Resolved] (KAFKA-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

2016-12-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4532?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4532. -- Resolution: Fixed Issue resolved by pull request 2250 [https://github.com/apache/kafka/pull/2250

[GitHub] kafka pull request #2250: KAFKA-4532: StateStores can be connected to the wr...

2016-12-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2250 --- 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-4532) StateStores can be connected to the wrong source topic resulting in incorrect metadata returned from IQ

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

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

2016-12-13 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4390; Replace MessageSet usage with client-side alternatives -- [...truncated 7920 lines...] kafka.utils.timer.TimerTaskListTest > testAll STARTED kafka.utils.timer.

[GitHub] kafka pull request #2252: HOTFIX: fix state transition stuck on rebalance

2016-12-13 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2252 HOTFIX: fix state transition stuck on rebalance This fixes a problem where the Kafka instance state transition gets stuck on rebalance. Also adjusts the test in QueryableStateIntegration test.

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

2016-12-13 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix Streams examples in documentation -- [...truncated 6277 lines...] kafka.api.PlaintextProducerSendTest > testSendOffset STARTED kafka.api.PlaintextProducerSen

[GitHub] kafka pull request #2242: KAFKA-4497: Fix the ByteBufferMessageSet.filterInt...

2016-12-13 Thread becketqin
Github user becketqin closed the pull request at: https://github.com/apache/kafka/pull/2242 --- 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 e

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2016-12-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746249#comment-15746249 ] ASF GitHub Bot commented on KAFKA-4497: --- Github user becketqin closed the pull reque

Re: [DISCUSS] KIP-90 Remove zkClient dependency from Streams

2016-12-13 Thread Gwen Shapira
Great idea, go for it :) On Tue, Dec 13, 2016 at 10:39 AM, Hojjat Jafarpour wrote: > Hi all, > > The following is a KIP for removing zkClient dependency from Streams. > Please check out the KIP page: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 90+-+Remove+zkClient+dependency+fro

Re: [DISCUSS] KIP-90 Remove zkClient dependency from Streams

2016-12-13 Thread Ismael Juma
Thanks for the KIP, Hojjat. It will be great for Streams apps not to require ZK access. Ismael On Tue, Dec 13, 2016 at 10:39 AM, Hojjat Jafarpour wrote: > Hi all, > > The following is a KIP for removing zkClient dependency from Streams. > Please check out the KIP page: > > https://cwiki.apache.

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Lorand Peter Kasler (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746368#comment-15746368 ] Lorand Peter Kasler commented on KAFKA-4477: We had encountered the same situa

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746393#comment-15746393 ] Jun Rao commented on KAFKA-4477: [~tdevoe], from the controller log, starting from 19:59:1

[jira] [Issue Comment Deleted] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom DeVoe updated KAFKA-4477: - Comment: was deleted (was: I was able to telnet from the controller node to the kafka port on node 1002 w

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746402#comment-15746402 ] Tom DeVoe commented on KAFKA-4477: -- I was able to telnet from the controller node to the

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Tom DeVoe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746403#comment-15746403 ] Tom DeVoe commented on KAFKA-4477: -- I was able to telnet from the controller node to the

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

2016-12-13 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746466#comment-15746466 ] Ismael Juma commented on KAFKA-4477: One question for the others that have reported th

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Lorand Peter Kasler (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746479#comment-15746479 ] Lorand Peter Kasler commented on KAFKA-4477: Yes, we recently upgraded and sta

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

2016-12-13 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-12-13 Thread Avi Flax
On 2016-11-28 13:47 (-0500), "Matthias J. Sax" wrote: > > I want to start a discussion about KIP-95: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-95%3A+Incremental+Batch+Processing+for+Kafka+Streams > > Looking forward to your feedback. Hi Matthias, I’d just like to share some fe

Kafka ACL's with SSL Protocol is not working

2016-12-13 Thread Raghu B
Hi All, I am trying to enable ACL's in my Kafka cluster with along with SSL Protocol. I tried with each and every parameters but no luck, so I need help to enable the SSL(without Kerberos) and I am attaching all the configuration details in this. Kindly Help me. *I tested SSL without ACL, it w

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-13 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746799#comment-15746799 ] Apurva Mehta commented on KAFKA-4477: - [~tdevoe], thanks for sharing all your extend b

  1   2   >