Re: Build failed in Jenkins: KafkaPreCommit #167

2015-07-29 Thread Ismael Juma
On Wed, Jul 29, 2015 at 4:52 PM, Guozhang Wang wangg...@gmail.com wrote: It is likely that we make two commits at roughly the same time that triggers two builds, how could this issue be resolve under this case? It should not happen in a typical configuration, but hard to tell what is wrong

[jira] [Resolved] (KAFKA-2382) KafkaConsumer seek methods should throw an exception when called for partitions not assigned to this consumer instance

2015-07-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2382?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-2382. -- Resolution: Invalid Ah, good call. Somehow I missed that, guess I didn't dig

Re: Build failed in Jenkins: KafkaPreCommit #167

2015-07-29 Thread Guozhang Wang
It is likely that we make two commits at roughly the same time that triggers two builds, how could this issue be resolve under this case? On Wed, Jul 29, 2015 at 2:23 AM, Ismael Juma ism...@juma.me.uk wrote: It looks like two builds are running in the same workspace at the same time somehow.

Re: Connection to zk shell on Kafka

2015-07-29 Thread Prabhjot Bharaj
Sure. It would be great if you could as well explain the reason why the absence of the jar creates this problem Also, I'm surprised that zookeeper that comes bundled with kafka 0.8.2 does not have the jline jar Regards, prabcs On Wed, Jul 29, 2015 at 10:45 PM, Chris Barlock barl...@us.ibm.com

[jira] [Commented] (KAFKA-2382) KafkaConsumer seek methods should throw an exception when called for partitions not assigned to this consumer instance

2015-07-29 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2382?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646338#comment-14646338 ] Jason Gustafson commented on KAFKA-2382: [~ewencp] I could be wrong, but I think

[jira] [Commented] (KAFKA-1913) App hungs when calls producer.send to wrong IP of Kafka broker

2015-07-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646373#comment-14646373 ] Ewen Cheslack-Postava commented on KAFKA-1913: -- [~igor.quickblox] I think

[jira] [Updated] (KAFKA-2214) kafka-reassign-partitions.sh --verify should return non-zero exit codes when reassignment is not completed yet

2015-07-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2214: Status: In Progress (was: Patch Available) Moved back to in progress since there are unaddressed

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Ismael Juma
Hi Gwen, Thanks for the feedback. Comments below. On Wed, Jul 29, 2015 at 6:40 PM, Gwen Shapira gshap...@cloudera.com wrote: The jira comment is a way for the committer to say thank you to people who were involved in the review process. If we just want to say thank you, then why not just

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Ismael Juma
Hi Parth, On Wed, Jul 29, 2015 at 6:50 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: +1 on Gwen¹s suggestion. Consider this as my thank you for all the reviews everyone has done in past and are going to do in future. Don¹t make me say thanks on every single commit. Introducing

Re: Kafka Consumer thoughts

2015-07-29 Thread Jiangjie Qin
I agree with Ewen that a single threaded model will be tricky to implement the same conventional semantic of async or Future. We just drafted the following wiki which explains our thoughts in LinkedIn on the new consumer API and threading model.

[jira] [Resolved] (KAFKA-2357) Update zookeeper.connect description in Kafka documentation

2015-07-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2357?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2357. -- Resolution: Fixed Update zookeeper.connect description in Kafka documentation

[jira] [Updated] (KAFKA-2357) Update zookeeper.connect description in Kafka documentation

2015-07-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2357?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2357: - Status: In Progress (was: Patch Available) Thanks for the patch, committed to the web site docs.

[jira] [Updated] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-29 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-2351: --- Status: Patch Available (was: In Progress) Brokers are having a problem shutting down

[jira] [Commented] (KAFKA-2300) Error in controller log when broker tries to rejoin cluster

2015-07-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2300?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646794#comment-14646794 ] Guozhang Wang commented on KAFKA-2300: -- [~junrao] would you like to take a look?

Re: Kafka Consumer thoughts

2015-07-29 Thread Jason Gustafson
I think this proposal matches pretty well with what user's intuitively expect the implementation to be. At a glance, I don't see any problems with doing the liveness detection in the background thread. It also has the advantage that the frequency of heartbeats (which controls how long rebalancing

Re: Kafka Consumer thoughts

2015-07-29 Thread Jay Kreps
Some comments on the proposal: I think we are conflating a number of things that should probably be addressed individually because they are unrelated. My past experience is that this always makes progress hard. The more we can pick apart these items the better: 1. threading model 2.

[jira] [Updated] (KAFKA-2300) Error in controller log when broker tries to rejoin cluster

2015-07-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2300?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2300: - Reviewer: Jun Rao Error in controller log when broker tries to rejoin cluster

Re: KAfka Mirror Maker

2015-07-29 Thread Jiangjie Qin
Mirror Maker does not have specific restrictions on cluster size. The error you saw was because consumer was not able to talk to the broker. Can you try to use kafka-console-consumer to consume some data from your source cluster and see if it works? It should be under KAFKA_HOME/bin/ Jiangjie

Re: Kafka Consumer thoughts

2015-07-29 Thread Neha Narkhede
Thanks Becket. Quick comment - there seem to be a bunch of images that the wiki refers to, but none loaded for me. Just making sure if its just me or can everyone not see the pictures? On Wed, Jul 29, 2015 at 12:00 PM, Jiangjie Qin j...@linkedin.com wrote: I agree with Ewen that a single

Re: Kafka Consumer thoughts

2015-07-29 Thread Guozhang Wang
The images load for me well. On Wed, Jul 29, 2015 at 1:10 PM, Neha Narkhede n...@confluent.io wrote: Thanks Becket. Quick comment - there seem to be a bunch of images that the wiki refers to, but none loaded for me. Just making sure if its just me or can everyone not see the pictures? On

Re: Kafka Consumer thoughts

2015-07-29 Thread Jiangjie Qin
Ah... My bad, forgot to change the URL link for pictures. Thanks for the quick response, Neha. It should be fixed now, can you try again? Jiangjie (Becket) Qin On Wed, Jul 29, 2015 at 1:10 PM, Neha Narkhede n...@confluent.io wrote: Thanks Becket. Quick comment - there seem to be a bunch of

Build failed in Jenkins: KafkaPreCommit #168

2015-07-29 Thread Apache Jenkins Server
See https://builds.apache.org/job/KafkaPreCommit/168/changes Changes: [cshapi] KAFKA-2100; Client Error doesn't preserve or display original server error code when it is an unknown code; Reviewed by Gwen, Guozhang and Ewen -- Started by an SCM change

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Gwen Shapira
I guess we see the reviewer part with different interpretations. What are the benefits you see of formalizing who gets mentioned as reviewer? On Wed, Jul 29, 2015 at 11:06 AM, Ismael Juma ism...@juma.me.uk wrote: Hi Gwen, Thanks for the feedback. Comments below. On Wed, Jul 29, 2015 at 6:40

Re: Kafka Consumer thoughts

2015-07-29 Thread Neha Narkhede
Works now. Thanks Becket! On Wed, Jul 29, 2015 at 1:19 PM, Jiangjie Qin j...@linkedin.com wrote: Ah... My bad, forgot to change the URL link for pictures. Thanks for the quick response, Neha. It should be fixed now, can you try again? Jiangjie (Becket) Qin On Wed, Jul 29, 2015 at 1:10 PM,

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Parth Brahmbhatt
+1 on Gwen¹s suggestion. Consider this as my thank you for all the reviews everyone has done in past and are going to do in future. Don¹t make me say thanks on every single commit. Introducing another process when the project has 50 PR open pretty much all the time is not really going to help.

[jira] [Updated] (KAFKA-2300) Error in controller log when broker tries to rejoin cluster

2015-07-29 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2300?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Flavio Junqueira updated KAFKA-2300: Attachment: KAFKA-2300.patch Uploading a patch for trunk. Error in controller log when

Re: Review Request 28096: Patch for KAFKA-313

2015-07-29 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28096/#review93489 --- Thanks for the patch and sorry for the long delay.

[jira] [Commented] (KAFKA-2100) Client Error doesn't preserve or display original server error code when it is an unknown code

2015-07-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646472#comment-14646472 ] Gwen Shapira commented on KAFKA-2100: - +1 and pushed to trunk. Thank you for your

[GitHub] kafka pull request: KAFKA-2300: Error in controller log when broke...

2015-07-29 Thread fpj
GitHub user fpj opened a pull request: https://github.com/apache/kafka/pull/102 KAFKA-2300: Error in controller log when broker tries to rejoin cluster You can merge this pull request into a Git repository by running: $ git pull https://github.com/fpj/kafka 2300

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Gwen Shapira
My two cents: The jira comment is a way for the committer to say thank you to people who were involved in the review process. It doesn't have any formal implications - the responsibility for committing good code is on the committer (thats the whole point). It doesn't even have informal

[jira] [Commented] (KAFKA-2300) Error in controller log when broker tries to rejoin cluster

2015-07-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2300?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646514#comment-14646514 ] ASF GitHub Bot commented on KAFKA-2300: --- GitHub user fpj opened a pull request:

[jira] [Commented] (KAFKA-1371) Ignore build output dirs

2015-07-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646911#comment-14646911 ] ASF GitHub Bot commented on KAFKA-1371: --- Github user sslavic closed the pull request

[GitHub] kafka pull request: KAFKA-1370 Added Gradle startup script for Win...

2015-07-29 Thread sslavic
Github user sslavic closed the pull request at: https://github.com/apache/kafka/pull/22 --- 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

[GitHub] kafka pull request: KAFKA-1371 Ignore build output dirs

2015-07-29 Thread sslavic
Github user sslavic closed the pull request at: https://github.com/apache/kafka/pull/21 --- 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

Re: Review Request 36858: Patch for KAFKA-2120

2015-07-29 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36858/ --- (Updated July 29, 2015, 10:57 p.m.) Review request for kafka. Bugs:

[GitHub] kafka pull request: Ignore gradle wrapper download directory

2015-07-29 Thread sslavic
Github user sslavic closed the pull request at: https://github.com/apache/kafka/pull/67 --- 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] [Created] (KAFKA-2384) Enable renaming commit message title in kafka-merge-pr.py

2015-07-29 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2384: Summary: Enable renaming commit message title in kafka-merge-pr.py Key: KAFKA-2384 URL: https://issues.apache.org/jira/browse/KAFKA-2384 Project: Kafka

[GitHub] kafka pull request: KAFKA-1375: Fix formatting in README.md

2015-07-29 Thread sslavic
Github user sslavic closed the pull request at: https://github.com/apache/kafka/pull/24 --- 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

Re: [DISCUSS] Reviewers in commit message

2015-07-29 Thread Ismael Juma
On Wed, Jul 29, 2015 at 7:38 PM, Gwen Shapira gshap...@cloudera.com wrote: I guess we see the reviewer part with different interpretations. Yes. As you know, Git was created for and initially used by the Linux Kernel. As such they were very influential in conventions, terminology and best

Re: [DISCUSS] KIP-28 - Add a transform client for data processing

2015-07-29 Thread Neha Narkhede
Prefer something that evokes stream processing on top of Kafka. And since I've heard many people conflate streaming with streaming video (I know, duh!), I'd vote for Kafka Streams or a maybe KStream. Thanks, Neha On Wed, Jul 29, 2015 at 6:08 PM, Jay Kreps j...@confluent.io wrote: Also, the

[GitHub] kafka pull request: MINOR: Fixed ConsumerRecord constructor javado...

2015-07-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/85 --- 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

Build failed in Jenkins: KafkaPreCommit #169

2015-07-29 Thread Apache Jenkins Server
See https://builds.apache.org/job/KafkaPreCommit/169/changes Changes: [wangguoz] MINOR: Fixed ConsumerRecord constructor javadoc -- Started by an SCM change Building remotely on ubuntu3 (Ubuntu ubuntu legacy-ubuntu) in workspace

[jira] [Updated] (KAFKA-2120) Add a request timeout to NetworkClient

2015-07-29 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2120?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-2120: --- Attachment: KAFKA-2120_2015-07-29_15:57:02.patch Add a request timeout to NetworkClient

[jira] [Commented] (KAFKA-2120) Add a request timeout to NetworkClient

2015-07-29 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646887#comment-14646887 ] Mayuresh Gharat commented on KAFKA-2120: Updated reviewboard

[jira] [Comment Edited] (KAFKA-1370) Gradle startup script for Windows

2015-07-29 Thread Stevo Slavic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1370?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13962801#comment-13962801 ] Stevo Slavic edited comment on KAFKA-1370 at 7/29/15 11:27 PM:

[jira] [Commented] (KAFKA-1370) Gradle startup script for Windows

2015-07-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1370?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646912#comment-14646912 ] ASF GitHub Bot commented on KAFKA-1370: --- Github user sslavic closed the pull request

Re: Kafka Consumer thoughts

2015-07-29 Thread Ismael Juma
Hi Jay, Good points. A few remarks below. On Wed, Jul 29, 2015 at 11:16 PM, Jay Kreps j...@confluent.io wrote: I suggest we focus on threading and the current event-loop style of api design since I think that is really the crux. Agreed. I think ultimately though what you need to think

Re: [DISCUSS] KIP-28 - Add a transform client for data processing

2015-07-29 Thread Gwen Shapira
Since it sounds like it is not a separate framework (like CopyCat) but rather a new client, it will be nice to follow existing convention. Producer, Consumer and Processor (or Transformer) make sense to me. Note that the way the API is currently described, people may want to use it inside Spark

Re: Review Request 35421: Patch for KAFKA-2026

2015-07-29 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/35421/#review93543 --- Ship it! Ship It! - Jiangjie Qin On June 13, 2015, 10:07 a.m.,

Re: Connection to zk shell on Kafka

2015-07-29 Thread Chris Barlock
I'm a user of Kafka/ZooKeeper not one of its developers, so I can't give you a technical explanation. I do agree that Kafka should ship the jline JAR if its zookeeper-shell depends on it. Chris From: Prabhjot Bharaj prabhbha...@gmail.com To: u...@zookeeper.apache.org,

Re: [DISCUSS] Partitioning in Kafka

2015-07-29 Thread Jiangjie Qin
Just my two cents. I think it might be OK to put this into Kafka if we agree that this might be a good use case for people who wants to use Kafka as temporary store for stream processing. At very least I don't see down side on this. Thanks, Jiangjie (Becket) Qin On Tue, Jul 28, 2015 at 3:41 AM,

[jira] [Commented] (KAFKA-2203) Get gradle build to work with Java 8

2015-07-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14645734#comment-14645734 ] Ismael Juma commented on KAFKA-2203: [~sslavic], does it work for you now that we no

[jira] [Updated] (KAFKA-2276) Initial patch for KIP-25

2015-07-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2276: --- Fix Version/s: (was: 0.9.0) 0.8.3 Initial patch for KIP-25

Re: Build failed in Jenkins: KafkaPreCommit #167

2015-07-29 Thread Ismael Juma
It looks like two builds are running in the same workspace at the same time somehow. Ismael On Wed, Jul 29, 2015 at 1:44 AM, Guozhang Wang wangg...@gmail.com wrote: Anyone knows how this Could not open buildscript class issue could happen and can we fix it on our side or it is a general

[jira] [Commented] (KAFKA-1913) App hungs when calls producer.send to wrong IP of Kafka broker

2015-07-29 Thread Igor Khomenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14645918#comment-14645918 ] Igor Khomenko commented on KAFKA-1913: -- For now I have to use the following utils to

[jira] [Comment Edited] (KAFKA-1913) App hungs when calls producer.send to wrong IP of Kafka broker

2015-07-29 Thread Igor Khomenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14645918#comment-14645918 ] Igor Khomenko edited comment on KAFKA-1913 at 7/29/15 11:53 AM:

[GitHub] kafka pull request: Fixed ConsumerRecord constructor javadoc

2015-07-29 Thread sslavic
GitHub user sslavic reopened a pull request: https://github.com/apache/kafka/pull/85 Fixed ConsumerRecord constructor javadoc Refactoring of ConsumerRecord made in https://github.com/apache/kafka/commit/0699ff2ce60abb466cab5315977a224f1a70a4da#diff-fafe8d3a3942f3c6394927881a9389b2

[GitHub] kafka pull request: Fixed ConsumerRecord constructor javadoc

2015-07-29 Thread sslavic
Github user sslavic closed the pull request at: https://github.com/apache/kafka/pull/85 --- 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

[DISCUSS] Reviewers in commit message

2015-07-29 Thread Ismael Juma
Hi all, As a general rule, we credit reviewers in the commit message. This is good. However, it is not clear to me if there are guidelines on who should be included as a reviewer (please correct me if I am wrong). I can think of a few options: 1. Anyone that commented on the patch (in the

Re: Review Request 36858: Patch for KAFKA-2120

2015-07-29 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36858/ --- (Updated July 29, 2015, 10:58 p.m.) Review request for kafka. Bugs:

[jira] [Updated] (KAFKA-2384) Enable renaming commit message title in kafka-merge-pr.py

2015-07-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2384: --- Issue Type: Improvement (was: Bug) Enable renaming commit message title in kafka-merge-pr.py

[jira] [Commented] (KAFKA-1375) Formatting for Running a task on a particular version of Scala paragraph in README.md is broken

2015-07-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1375?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646907#comment-14646907 ] ASF GitHub Bot commented on KAFKA-1375: --- Github user sslavic closed the pull request

Re: [DISCUSS] KIP-28 - Add a transform client for data processing

2015-07-29 Thread Jay Kreps
Also, the most important part of any prototype, we should have a name for this producing-consumer-thingamgigy: Various ideas: - Kafka Streams - KStream - Kafka Streaming - The Processor API - Metamorphosis - Transformer API - Verwandlung For my part I think what people are trying to do is stream

Re: Kafka Consumer thoughts

2015-07-29 Thread Jiangjie Qin
Thanks for the comments Jason and Jay. Jason, I had the same concern for producer's callback as well before, but it seems to be fine from some callbacks I wrote - user can always pass in object in the constructor if necessary for synchronization. Jay, I agree that the current API might be fine

[GitHub] kafka pull request: MINOR - Fix typo in ReplicaVerificationTool ou...

2015-07-29 Thread ottomata
GitHub user ottomata opened a pull request: https://github.com/apache/kafka/pull/101 MINOR - Fix typo in ReplicaVerificationTool output You can merge this pull request into a Git repository by running: $ git pull https://github.com/ottomata/kafka trunk Alternatively you can

[jira] [Commented] (KAFKA-2383) kafka-env should be separated between client and server

2015-07-29 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2383?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646069#comment-14646069 ] Sriharsha Chintalapani commented on KAFKA-2383: --- [~AdamWesterman]