[jira] [Commented] (KAFKA-2426) A Kafka node tries to connect to itself through its advertised hostname

2016-01-06 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-2426?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086853#comment-15086853 ] Mikaƫl Cluseau commented on KAFKA-2426: --- Hi Matt, yes, the only way to make it work

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

2016-01-06 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2929: Migrate duplicate error mapping functionality -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu3 (Ubuntu ubun

[jira] [Commented] (KAFKA-2929) Migrate server side error mapping functionality

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

[GitHub] kafka pull request: KAFKA-2929: Migrate duplicate error mapping fu...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/616 --- 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 enabl

[jira] [Updated] (KAFKA-2929) Migrate server side error mapping functionality

2016-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2929: - Resolution: Fixed Fix Version/s: 0.9.1.0 Status: Resolved (was:

[jira] [Updated] (KAFKA-3075) java.util.HashMap cannot be cast to scala.collection.immutable.Map When using ZookeeperConsumerConnector.commitOffsets

2016-01-06 Thread Pengwei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pengwei updated KAFKA-3075: --- Description: When using java api's commit offset : public void commitOffsets(Map offsetsToCommit, boolean r

[jira] [Updated] (KAFKA-3075) java.util.HashMap cannot be cast to scala.collection.immutable.Map When using ZookeeperConsumerConnector.commitOffsets

2016-01-06 Thread Pengwei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pengwei updated KAFKA-3075: --- Description: When using java api's commit offset : public void commitOffsets(Map offsetsToCommit, boolean r

[jira] [Created] (KAFKA-3075) java.util.HashMap cannot be cast to scala.collection.immutable.Map When using ZookeeperConsumerConnector.commitOffsets

2016-01-06 Thread Pengwei (JIRA)
Pengwei created KAFKA-3075: -- Summary: java.util.HashMap cannot be cast to scala.collection.immutable.Map When using ZookeeperConsumerConnector.commitOffsets Key: KAFKA-3075 URL: https://issues.apache.org/jira/browse/KA

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086717#comment-15086717 ] Jun Rao commented on KAFKA-3068: There are 2 possible ways of configuring bootstrap server

Re: [DISCUSS] KIP-36 - Rack aware replica assignment

2016-01-06 Thread Allen Wang
Updated KIP according to Jun's comment and included changes to TMR. On Tue, Jan 5, 2016 at 5:59 PM, Jun Rao wrote: > Hi, Allen, > > A couple of minor comments on the KIP. > > 1. The version of the broker JSON string says 2. It should be 3. > > 2. The new version of UpdateMetadataRequest should b

Re: KIP-41: KafkaConsumer Max Records

2016-01-06 Thread Gwen Shapira
I like the fair-consumption approach you chose - "pull as many records as possible from each partition in a similar round-robin fashion", it is very intuitive and close enough to fair. Overall, I'm +1 on the KIP. But you'll need a formal vote :) On Wed, Jan 6, 2016 at 6:05 PM, Jason Gustafson wr

Re: KIP-41: KafkaConsumer Max Records

2016-01-06 Thread Jason Gustafson
Thanks for the suggestion, Ismael. I updated the KIP. -Jason On Wed, Jan 6, 2016 at 6:57 AM, Ismael Juma wrote: > Thanks Jason. I read the KIP and it makes sense to me. A minor suggestion: > in the "Ensuring Fair Consumption" section, there are 3 paragraphs with 2 > examples (2 partitions with

[jira] [Resolved] (KAFKA-3074) Regex sink connector support

2016-01-06 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-3074. - Resolution: Duplicate > Regex sink connector support > > >

[jira] [Created] (KAFKA-3074) Regex sink connector support

2016-01-06 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-3074: Summary: Regex sink connector support Key: KAFKA-3074 URL: https://issues.apache.org/jira/browse/KAFKA-3074 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-3073) KafkaConnect should support regular expression for topics

2016-01-06 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-3073: --- Summary: KafkaConnect should support regular expression for topics Key: KAFKA-3073 URL: https://issues.apache.org/jira/browse/KAFKA-3073 Project: Kafka Issue T

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086580#comment-15086580 ] Ismael Juma commented on KAFKA-3068: It does to me. > NetworkClient may connect to a

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

2016-01-06 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2874; shutdown ZK process reliably [junrao] KAFKA-2072; Replace StopReplica Request/Response with their [me] KAFKA-3022: Deduplicate common project configurations --

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

2016-01-06 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2937; Disable the leaderIsr check if the topic is to be deleted. [wangguoz] KAFKA-3016: phase-2. stream join implementations [junrao] KAFKA-2850; Fix SSL invalid endpoint validation test --

[jira] [Updated] (KAFKA-3022) Deduplicate common project configurations

2016-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3022?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3022: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved

[jira] [Commented] (KAFKA-3022) Deduplicate common project configurations

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

[GitHub] kafka pull request: KAFKA-3022: Deduplicate common project configu...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/712 --- 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 enabl

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086554#comment-15086554 ] Jason Gustafson commented on KAFKA-3068: [~ijuma],[~enothereska] Introducing a not

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

2016-01-06 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: KAFKA-2072: Replace StopReplica Request/Respon...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/196 --- 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 enabl

[jira] [Commented] (KAFKA-2072) Add StopReplica request/response to o.a.k.common.requests and replace the usage in core module

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

[jira] [Updated] (KAFKA-2072) Add StopReplica request/response to o.a.k.common.requests and replace the usage in core module

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2072?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2072: --- Resolution: Fixed Fix Version/s: 0.9.1.0 Status: Resolved (was: Patch Available) Issue reso

[jira] [Created] (KAFKA-3072) Client should only allow commits for assigned partitions

2016-01-06 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-3072: -- Summary: Client should only allow commits for assigned partitions Key: KAFKA-3072 URL: https://issues.apache.org/jira/browse/KAFKA-3072 Project: Kafka Is

[jira] [Commented] (KAFKA-2874) zookeeper-server-stop.sh may fail to shutdown ZK and/or may stop unrelated processes

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

[jira] [Resolved] (KAFKA-2874) zookeeper-server-stop.sh may fail to shutdown ZK and/or may stop unrelated processes

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-2874. Resolution: Fixed Fix Version/s: 0.9.1.0 Issue resolved by pull request 573 [https://github.com/apach

[GitHub] kafka pull request: KAFKA-2874: shutdown ZK process reliably

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/573 --- 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 enabl

[jira] [Commented] (KAFKA-2887) TopicMetadataRequest creates topic if it does not exist

2016-01-06 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086470#comment-15086470 ] Mayuresh Gharat commented on KAFKA-2887: [~hachikuji] just wanted to know if we ha

[jira] [Commented] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

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

[GitHub] kafka pull request: KAFKA-2850: Fix SSL invalid endpoint validatio...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/546 --- 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 enabl

[jira] [Resolved] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-2850. Resolution: Fixed Fix Version/s: 0.9.1.0 Issue resolved by pull request 546 [https://github.com/apach

[jira] [Resolved] (KAFKA-3016) Add KStream-KStream window joins

2016-01-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3016?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3016. -- Resolution: Fixed Fix Version/s: 0.9.1.0 Issue resolved by pull request 737 [https://gith

[jira] [Commented] (KAFKA-3016) Add KStream-KStream window joins

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

[GitHub] kafka pull request: KAFKA-3016: phase-2. stream join implementatio...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/737 --- 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 enabl

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

2016-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-1901; Added error handling when no file exists in .git/refs/heads. -- [...truncated 1782 lines...] kafka.admin.AddPartitionsTest > testTopicDoesNotExist PASSED ka

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

2016-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-1901; Added error handling when no file exists in .git/refs/heads. -- [...truncated 6951 lines...] org.apache.kafka.connect.storage.KafkaConfigStorageTest > testRes

[jira] [Commented] (KAFKA-2937) Topics marked for delete in Zookeeper may become undeletable

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

[GitHub] kafka pull request: KAFKA-2937 : Disable the leaderIsr check if th...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/729 --- 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 enabl

[jira] [Updated] (KAFKA-2937) Topics marked for delete in Zookeeper may become undeletable

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2937: --- Resolution: Fixed Fix Version/s: 0.9.0.1 Status: Resolved (was: Patch Available) Issue reso

[jira] [Commented] (KAFKA-2937) Topics marked for delete in Zookeeper may become undeletable

2016-01-06 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086375#comment-15086375 ] Jun Rao commented on KAFKA-2937: The situation that [~mgharat] described can indeed happen

[jira] [Commented] (KAFKA-3070) SASL unit tests dont work with IBM JDK

2016-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086252#comment-15086252 ] ASF GitHub Bot commented on KAFKA-3070: --- GitHub user rajinisivaram opened a pull req

[jira] [Commented] (KAFKA-1901) Move Kafka version to be generated in code by build (instead of in manifest)

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

[jira] [Updated] (KAFKA-3070) SASL unit tests dont work with IBM JDK

2016-01-06 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3070: -- Status: Patch Available (was: Open) > SASL unit tests dont work with IBM JDK >

[GitHub] kafka pull request: KAFKA-1901: follow-up on KAFKA-1901; Added err...

2016-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/209 --- 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 enabl

[GitHub] kafka pull request: KAFKA-3070: Use IBM Kerberos module for SASL t...

2016-01-06 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/738 KAFKA-3070: Use IBM Kerberos module for SASL tests if running on IBM JDK Use IBM Kerberos module and properties for SASL tests if using IBM JRE You can merge this pull request into a Git reposi

[jira] [Resolved] (KAFKA-1059) Improve the patch review tool to use OAuth for JIRA access

2016-01-06 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1059?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke resolved KAFKA-1059. Resolution: Won't Fix Assignee: Grant Henke (was: Neha Narkhede) Closing because the patch re

[jira] [Commented] (KAFKA-664) Kafka server threads die due to OOME during long running test

2016-01-06 Thread vijay bhaskar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-664?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15086165#comment-15086165 ] vijay bhaskar commented on KAFKA-664: - Thanks for the suggestion, Ismael. I have create

[jira] [Created] (KAFKA-3071) Kafka Server 0.8.2 ERROR OOME with siz

2016-01-06 Thread vijay bhaskar (JIRA)
vijay bhaskar created KAFKA-3071: Summary: Kafka Server 0.8.2 ERROR OOME with siz Key: KAFKA-3071 URL: https://issues.apache.org/jira/browse/KAFKA-3071 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-2975) The newtorkClient should request a metadata update after it gets an error in the handleResponse()

2016-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085926#comment-15085926 ] ASF GitHub Bot commented on KAFKA-2975: --- Github user MayureshGharat closed the pull

[GitHub] kafka pull request: KAFKA-2975

2016-01-06 Thread MayureshGharat
Github user MayureshGharat closed the pull request at: https://github.com/apache/kafka/pull/658 --- 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

[GitHub] kafka pull request: KAFKA-3016: phase-2. stream join implementatio...

2016-01-06 Thread ymatsuda
GitHub user ymatsuda opened a pull request: https://github.com/apache/kafka/pull/737 KAFKA-3016: phase-2. stream join implementations @guozhangwang You can merge this pull request into a Git repository by running: $ git pull https://github.com/ymatsuda/kafka windowed_join2 A

[jira] [Commented] (KAFKA-3016) Add KStream-KStream window joins

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

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-06 Thread Jun Rao
Hi, Jiangjie, 52. Replacing MessageSet with o.a.k.common.record will be ideal. Unfortunately, we use MessageSet in SimpleConsumer, which is part of the public api. Replacing MessageSet with o.a.k.common.record will be an incompatible api change. So, we probably should do this after we deprecate Si

[jira] [Commented] (KAFKA-3069) Fix recursion in ZkSecurityMigrator

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

[GitHub] kafka pull request: KAFKA-3069: Fix recursion in ZkSecurityMigrato...

2016-01-06 Thread fpj
GitHub user fpj opened a pull request: https://github.com/apache/kafka/pull/736 KAFKA-3069: Fix recursion in ZkSecurityMigrator I'm also fixing a bug in the testChroot test case. You can merge this pull request into a Git repository by running: $ git pull https://github.com/fpj

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-06 Thread Becket Qin
Hey Jay, I agree with your comments on the KIP page. As you said it is important to document what are the changes and why we made those decisions. I have added summaries to the page to help readers understand the decision we made more easily and also try to avoid being too verbose. I also clari

[jira] [Updated] (KAFKA-2948) Kafka producer does not cope well with topic deletions

2016-01-06 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2948?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2948: -- Status: Patch Available (was: Open) > Kafka producer does not cope well with topic deletions >

[jira] [Updated] (KAFKA-3065) Prevent unbounded growth of RecordAccumulator#batches in Kafka producer

2016-01-06 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3065?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3065: -- Status: Patch Available (was: Open) PR deletes unused entries from RecordAccumulator#batches. T

[jira] [Created] (KAFKA-3070) SASL unit tests dont work with IBM JDK

2016-01-06 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-3070: - Summary: SASL unit tests dont work with IBM JDK Key: KAFKA-3070 URL: https://issues.apache.org/jira/browse/KAFKA-3070 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-06 Thread Becket Qin
Thanks a lot for the careful reading, Jun. Please see inline replies. > On Jan 6, 2016, at 3:24 AM, Jun Rao wrote: > > Jiangjie, > > Thanks for the updated KIP. Overall, a +1 on the proposal. A few minor > comments on the KIP. > > KIP-32: > 50. 6.c says "The log rolling has to depend on the e

[jira] [Commented] (KAFKA-3065) Prevent unbounded growth of RecordAccumulator#batches in Kafka producer

2016-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3065?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085694#comment-15085694 ] ASF GitHub Bot commented on KAFKA-3065: --- GitHub user rajinisivaram opened a pull req

[GitHub] kafka pull request: KAFKA-3065: Remove unused topic partitions fro...

2016-01-06 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/735 KAFKA-3065: Remove unused topic partitions from RecordAccumulator Removes unused topic partitions from RecordAccumulator#batches to prevent the map growing indefinitely. Replaces CopyOnWriteMap

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085688#comment-15085688 ] Ismael Juma commented on KAFKA-3068: Yeah. Or expire brokers based on time. Let's see

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085682#comment-15085682 ] Eno Thereska commented on KAFKA-3068: - [~ijuma]: a slightly different solution would b

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085663#comment-15085663 ] Ismael Juma commented on KAFKA-3068: [~enothereska], I agree that a better solution wo

[jira] [Comment Edited] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085618#comment-15085618 ] Eno Thereska edited comment on KAFKA-3068 at 1/6/16 3:15 PM: -

[jira] [Updated] (KAFKA-2757) Consolidate BrokerEndPoint and EndPoint

2016-01-06 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-2757: Assignee: (was: Jeff Holoman) > Consolidate BrokerEndPoint and EndPoint > --

[jira] [Commented] (KAFKA-3038) Speeding up partition reassignment after broker failure

2016-01-06 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085628#comment-15085628 ] Eno Thereska commented on KAFKA-3038: - [~fpj]: makes sense, thanks > Speeding up part

[jira] [Assigned] (KAFKA-3069) Fix recursion in ZkSecurityMigrator

2016-01-06 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Flavio Junqueira reassigned KAFKA-3069: --- Assignee: Flavio Junqueira > Fix recursion in ZkSecurityMigrator > --

[jira] [Created] (KAFKA-3069) Fix recursion in ZkSecurityMigrator

2016-01-06 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-3069: --- Summary: Fix recursion in ZkSecurityMigrator Key: KAFKA-3069 URL: https://issues.apache.org/jira/browse/KAFKA-3069 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3068) NetworkClient may connect to a different Kafka cluster than originally configured

2016-01-06 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3068?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085618#comment-15085618 ] Eno Thereska commented on KAFKA-3068: - [~ijuma], [~hachikuji]: I think a proper soluti

Re: KIP-41: KafkaConsumer Max Records

2016-01-06 Thread Ismael Juma
Thanks Jason. I read the KIP and it makes sense to me. A minor suggestion: in the "Ensuring Fair Consumption" section, there are 3 paragraphs with 2 examples (2 partitions with 100 max.poll.records and 3 partitions with 30 max.poll.records). I think you could simplify this by using one of the examp

Re: ZkSecurityMigrator incorrectly applies ACLs to entire ZooKeeper tree?

2016-01-06 Thread Flavio Junqueira
You're right, we should set the root ACL separately instead of starting the recursion on the root. One other reason for doing this is that there is at least one persistent branch we don't want to secure (ConsumersPath). I'll create a jira for this and fix it. -Flavio > On 06 Jan 2016, at 14:09

RE: ZkSecurityMigrator incorrectly applies ACLs to entire ZooKeeper tree?

2016-01-06 Thread Matthew Bruce
Hi Flavio, We're not using a chroot for Kafka itself as the ensemble is (mostly) not shared. We do have 1 or two extra branches that are used for some custom programs that use our Kafka infrastructure (each of these branches are used as a chroot). In hindsight using a separate chroot for Kafk

[jira] [Commented] (KAFKA-664) Kafka server threads die due to OOME during long running test

2016-01-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-664?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085427#comment-15085427 ] Ismael Juma commented on KAFKA-664: --- Vijay, you should probably file a new issue and incl

[jira] [Commented] (KAFKA-664) Kafka server threads die due to OOME during long running test

2016-01-06 Thread vijay bhaskar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-664?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085389#comment-15085389 ] vijay bhaskar commented on KAFKA-664: - I am seeing similar errors on Kafka which is run

[GitHub] kafka pull request: Grammar on README.md

2016-01-06 Thread simplyianm
Github user simplyianm closed the pull request at: https://github.com/apache/kafka/pull/588 --- 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-3038) Speeding up partition reassignment after broker failure

2016-01-06 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15085300#comment-15085300 ] Flavio Junqueira commented on KAFKA-3038: - You don't really need to batch with mul

Re: ZkSecurityMigrator incorrectly applies ACLs to entire ZooKeeper tree?

2016-01-06 Thread Flavio Junqueira
Hi Matthew, If you're sharing a ZK ensemble and you have a specific path for the Kafka znodes, then you need to use a chroot for this. Just pass it along with the connect string: http://zookeeper.apache.org/doc/r3.4.6/zookeeperProgrammers.html#ch_zkSessions