Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Jaikiran Pai
> On Jan. 29, 2015, 6:50 a.m., Gwen Shapira wrote: > > We added --override option to KafkaServer that allows overriding default > > configuration from commandline. > > I believe that just changing the shell script to include --override > > log.dir=${KAFKA_HOME}/data > > may be enough? > > > >

Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Jaikiran Pai
> On Jan. 29, 2015, 6:50 a.m., Gwen Shapira wrote: > > We added --override option to KafkaServer that allows overriding default > > configuration from commandline. > > I believe that just changing the shell script to include --override > > log.dir=${KAFKA_HOME}/data > > may be enough? > > > >

Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30403/#review70168 --- We added --override option to KafkaServer that allows overriding def

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-01-28 Thread Bhavesh Mistry
Hi Jiangjie, I just wanted to let you know about our use case and stress the point that local data center broker cluster have fewer partitions than the destination offline broker cluster. Just because we do the batch pull from CAMUS and in order to drain data faster than the injection rate (from f

Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Jaikiran Pai
> On Jan. 29, 2015, 6:32 a.m., Sriharsha Chintalapani wrote: > > core/src/main/scala/kafka/server/KafkaConfig.scala, line 149 > > > > > > I am not sure all of these code changes necessary. Instead change the > > defau

Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30403/#review70163 --- core/src/main/scala/kafka/server/KafkaConfig.scala

Changing the default Kafka data log directory (was Re: What to do when file.rename fails?)

2015-01-28 Thread Jaikiran Pai
I have created a JIRA for this proposed change https://issues.apache.org/jira/browse/KAFKA-1906 and uploaded a patch for review https://reviews.apache.org/r/30403/ -Jaikiran On Tuesday 27 January 2015 02:36 AM, Jay Kreps wrote: Having a relative path and keeping data under /data in the kafka

Re: Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30403/ --- (Updated Jan. 29, 2015, 6:24 a.m.) Review request for kafka. Changes ---

[jira] [Updated] (KAFKA-1906) Default the Kafka log directory to a relative path within the Kafka installation directory

2015-01-28 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1906: Attachment: KAFKA-1906.patch > Default the Kafka log directory to a relative path within the Kafka

[jira] [Commented] (KAFKA-1906) Default the Kafka log directory to a relative path within the Kafka installation directory

2015-01-28 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296435#comment-14296435 ] jaikiran pai commented on KAFKA-1906: - Created reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-1906) Default the Kafka log directory to a relative path within the Kafka installation directory

2015-01-28 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1906: Status: Patch Available (was: Open) > Default the Kafka log directory to a relative path within the

Review Request 30403: Patch for KAFKA-1906

2015-01-28 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30403/ --- Review request for kafka. Bugs: KAFKA-1906 https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-1906) Default the Kafka log directory to a relative path within the Kafka installation directory

2015-01-28 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-1906: --- Summary: Default the Kafka log directory to a relative path within the Kafka installation directory Key: KAFKA-1906 URL: https://issues.apache.org/jira/browse/KAFKA-1906

[VOTE] 0.8.2.0 Candidate 3

2015-01-28 Thread Jun Rao
This is the third candidate for release of Apache Kafka 0.8.2.0. Release Notes for the 0.8.2.0 release https://people.apache.org/~junrao/kafka-0.8.2.0-candidate3/RELEASE_NOTES.html *** Please download, test and vote by Saturday, Jan 31, 11:30pm PT Kafka's KEYS file containing PGP keys we use to

Re: [DISCUSSION] generate explicit error/failing metrics

2015-01-28 Thread Guozhang Wang
I think this is more of a tooling issue that the new consumer may not directly resolve. On the other hand, the ConsumerOffsetChecker tool will return for each partition the consumer's current offset as well as the log end offset, if some partitions are not owned by consumers, their "owner" field wi

[jira] [Comment Edited] (KAFKA-1903) Zk Expiration causes controller deadlock

2015-01-28 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296354#comment-14296354 ] Sriharsha Chintalapani edited comment on KAFKA-1903 at 1/29/15 4:19 AM:

[jira] [Commented] (KAFKA-1903) Zk Expiration causes controller deadlock

2015-01-28 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296354#comment-14296354 ] Sriharsha Chintalapani commented on KAFKA-1903: --- [~rawind] There are quite a

[jira] [Created] (KAFKA-1905) KafkaProducer's performance could be halved when MaxInFlightRequest is set to 1

2015-01-28 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-1905: --- Summary: KafkaProducer's performance could be halved when MaxInFlightRequest is set to 1 Key: KAFKA-1905 URL: https://issues.apache.org/jira/browse/KAFKA-1905 Project:

[jira] [Updated] (KAFKA-1861) Publishing kafka-client:test in order to utilize the helper utils in TestUtils

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1861?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1861: --- Resolution: Fixed Fix Version/s: 0.8.2 Status: Resolved (was: Patch Available) Thanks for t

[jira] [Assigned] (KAFKA-1860) File system errors are not detected unless Kafka tries to write

2015-01-28 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat reassigned KAFKA-1860: -- Assignee: Mayuresh Gharat (was: Guozhang Wang) > File system errors are not detected u

Request to be added to this mailing list

2015-01-28 Thread Abhishek Nigam

Jenkins build is back to normal : Kafka-trunk #382

2015-01-28 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296205#comment-14296205 ] Jun Rao commented on KAFKA-1729: Joel, thanks for the patch. +1 and committed to 0.8.2. Th

Re: Review Request 30355: 0.8.2 javaapi offset management API improvements

2015-01-28 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30355/#review70133 --- Ship it! Ship It! - Jun Rao On Jan. 28, 2015, 3:09 a.m., Joel Ko

[jira] [Updated] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1902: --- Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the reviews. Committed to 0.8.2

[jira] [Updated] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1904: - Fix Version/s: (was: 0.8.2) 0.8.3 > run sanity failed test > --

[jira] [Commented] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296179#comment-14296179 ] Joe Stein commented on KAFKA-1904: -- 0.8.2 works => Total failures count : 0 > run sanity

[jira] [Commented] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14296176#comment-14296176 ] Jun Rao commented on KAFKA-1904: Does this happen on 0.8.2 too? > run sanity failed test

[jira] [Updated] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1904: - Attachment: run_sanity.log.gz attached run on trunk, 1 failure lots of skips > run sanity failed test > -

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-01-28 Thread Jiangjie Qin
I’ve updated the KIP page. Feedbacks are welcome. Regarding the simple mirror maker design. I thought over it and have some worries: There are two things that might worth thinking: 1. One of the enhancement to mirror maker is adding a message handler to do things like reformatting. I think we migh

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

2015-01-28 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295614#comment-14295614 ] Joe Stein commented on KAFKA-1901: -- The same way you are suggesting for the brokers to ha

Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints

2015-01-28 Thread Gwen Shapira
Bumping :) If there are no objections, I'd like to go with the following: 1. Do not support javaapi (SimpleConsumer) with dependency on versions higher than 0.8.2. Existing clients will keep working. 2. The configuration parameter for upgrades will be inter.broker.protocol.version={0.8.2.0, 0.8.3

[jira] [Updated] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-01-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1809: Attachment: KAFKA-1809_2015-01-28_10:26:22.patch > Refactor brokers to allow listening on multiple p

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

2015-01-28 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295566#comment-14295566 ] Joel Koshy commented on KAFKA-1901: --- Can you clarify what you mean by metadata fetch for

[jira] [Commented] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-01-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295565#comment-14295565 ] Gwen Shapira commented on KAFKA-1809: - Updated reviewboard https://reviews.apache.org/

Re: Review Request 28769: Patch for KAFKA-1809

2015-01-28 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/ --- (Updated Jan. 28, 2015, 6:26 p.m.) Review request for kafka. Bugs: KAFKA-1809

Re: Review Request 30321: Patch for kafka-1902

2015-01-28 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30321/#review70031 --- Ship it! core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala

[jira] [Commented] (KAFKA-1861) Publishing kafka-client:test in order to utilize the helper utils in TestUtils

2015-01-28 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295551#comment-14295551 ] Manikumar Reddy commented on KAFKA-1861: [~junrao] we can include this is 0.8.2 RC

[jira] [Updated] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-01-28 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1729: -- Description: (was: [~junrao] before you roll out another 0.8.2 rc can you review this?) > add doc f

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295526#comment-14295526 ] Jason Rosenberg commented on KAFKA-1902: LGTM > fix MetricName so that Yammer rep

[jira] [Updated] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-01-28 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1729: -- Reviewer: Jun Rao Description: [~junrao] before you roll out another 0.8.2 rc can you review this

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Kyle Banker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295507#comment-14295507 ] Kyle Banker commented on KAFKA-1902: I've just verified the patch against Graphite in

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Kyle Banker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295483#comment-14295483 ] Kyle Banker commented on KAFKA-1902: I'll run the patch and report back. > fix Metric

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Kyle Banker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295454#comment-14295454 ] Kyle Banker commented on KAFKA-1902: This looks great, [~junrao]! > fix MetricName so

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295435#comment-14295435 ] Jun Rao commented on KAFKA-1902: It seems that the ganglia reporter replaces anything othe

[jira] [Updated] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1902: --- Attachment: kafka-1902_2015-01-28_09:23:51.patch > fix MetricName so that Yammer reporter can work correctly >

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295420#comment-14295420 ] Jun Rao commented on KAFKA-1902: Updated reviewboard https://reviews.apache.org/r/30321/di

Re: Review Request 30321: Patch for kafka-1902

2015-01-28 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30321/ --- (Updated Jan. 28, 2015, 5:23 p.m.) Review request for kafka. Bugs: kafka-1902

[jira] [Commented] (KAFKA-1902) fix MetricName so that Yammer reporter can work correctly

2015-01-28 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295359#comment-14295359 ] Jason Rosenberg commented on KAFKA-1902: For the record, at Square, we do use '.''

Re: Review Request 30321: Patch for kafka-1902

2015-01-28 Thread Eric Olander
> On Jan. 28, 2015, 2:34 a.m., Eric Olander wrote: > > core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala, line 66 > > > > > > val scope = tagsName.map {t => nameBuilder.append(",").append(t)}.orNull > > Eric Ol

[jira] [Commented] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14295302#comment-14295302 ] Gwen Shapira commented on KAFKA-1904: - [~joestein] - I guess my announcement that it w

Re: Review Request 30321: Patch for kafka-1902

2015-01-28 Thread Eric Olander
> On Jan. 28, 2015, 2:34 a.m., Eric Olander wrote: > > core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala, line 66 > > > > > > val scope = tagsName.map {t => nameBuilder.append(",").append(t)}.orNull Sorry - I h

[jira] [Updated] (KAFKA-1904) run sanity failed test

2015-01-28 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1904: - Summary: run sanity failed test (was: run sanity failured test) > run sanity failed test > --

[jira] [Created] (KAFKA-1904) run sanity failured test

2015-01-28 Thread Joe Stein (JIRA)
Joe Stein created KAFKA-1904: Summary: run sanity failured test Key: KAFKA-1904 URL: https://issues.apache.org/jira/browse/KAFKA-1904 Project: Kafka Issue Type: Bug Reporter: Joe Stei

[jira] [Created] (KAFKA-1903) Zk Expiration causes controller deadlock

2015-01-28 Thread yufeng.chen (JIRA)
yufeng.chen created KAFKA-1903: -- Summary: Zk Expiration causes controller deadlock Key: KAFKA-1903 URL: https://issues.apache.org/jira/browse/KAFKA-1903 Project: Kafka Issue Type: Bug