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

2015-01-26 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: -- Attachment: KAFKA-1782-doc-v3.patch Incorporated comments. One issue we have with the current linked wi

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

2015-01-26 Thread Gwen Shapira
Hi Kafka Devs, While reviewing the patch for KAFKA-1809, we came across two questions that we are interested in hearing the community out on. 1. This patch changes the Broker class and adds a new class BrokerEndPoint that behaves like the previous broker. While technically kafka.cluster.Broker i

Re: Review Request 28769: Patch for KAFKA-1809

2015-01-26 Thread Gwen Shapira
> On Jan. 23, 2015, 1:57 a.m., Jun Rao wrote: > > Thanks for the patch. Looks promising. Some comments. > > > > 1. I overlooked this when I suggested the new broker format in ZK. This > > means that we will need to upgrade all consumer clients before we can turn > > on the flag of using the ne

Re: [DISCUSS] KIPs

2015-01-26 Thread Gwen Shapira
Sorry for late response, Magnus. See my comments inline: On Fri, Jan 23, 2015 at 7:31 AM, Magnus Edenhill wrote: > Wouldn't it make sense to move away from these rich binary broker > descriptors ({ host, port, proto }) > (which require protocol churning on change), and simply use URIs instead? W

Re: Kafka Out of Memory error

2015-01-26 Thread Pranay Agarwal
Thanks a lot Gwen. I bumped up the JVM to 1g on the consumer side and it works :) All the consumer belong to the same group and I am using the High level group API to consume from the kafka. It seems there is some initial meta data exchange or something about all the partitions are sent to all the

Re: Review Request 30259: Add static code coverage reporting capability

2015-01-26 Thread Eric Olander
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30259/#review69727 --- This is a nice improvement to the project. Thanks! core/src/main/

Re: What to do when file.rename fails?

2015-01-26 Thread Gwen Shapira
Our logs are called .log. There are sysadmins that automatically clean old log files with a small cron job that does: find / -name "*.log" -ctime +30 -exec rm{} I suspect that allowing Kafka admins to choose a different suffix may also help with cases where files suddenly disappear. (i.e we regula

Re: What to do when file.rename fails?

2015-01-26 Thread Gwen Shapira
I think that most packages already default log.dir to something more reasonable. On Mon, Jan 26, 2015 at 1:06 PM, Jay Kreps wrote: > Having a relative path and keeping data under /data in the kafka distro > would make sense. This would require some reworking of the shell scripts, > though, as I t

Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Otis Gospodnetic
Hi, Don't use Graphite, so I don't know. Kyle, maybe you can share more info? What do you mean by "reported to Yammer" for example? And when you say Yammer/Graphite, are you trying to say that you are using the Graphite Reporter? If so, can you try other Yammer Reporters and see if there is a m

Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Jun Rao
Kyle, Also, which reporter are you using for Graphite? Thanks, Jun On Mon, Jan 26, 2015 at 4:08 PM, Kyle Banker wrote: > This is still preliminary, but it looks as if the change to metric names > for per-topic metrics (bytes/messages in/out) is preventing these metrics > from being reported t

[jira] [Commented] (KAFKA-1897) Enhance MockProducer for more sophisticated tests

2015-01-26 Thread Navina Ramesh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1897?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292748#comment-14292748 ] Navina Ramesh commented on KAFKA-1897: -- 1. Yes. We can use the Cluster constructor.

Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Jun Rao
Hmm, that's not the intention. The per-topic mbeans are definitely registered by Yammer. So, not sure why it's not reported to Graphite. Otis, Vladimir, Do you guys know? Thanks, Jun On Mon, Jan 26, 2015 at 4:08 PM, Kyle Banker wrote: > This is still preliminary, but it looks as if the cha

Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Kyle Banker
This is still preliminary, but it looks as if the change to metric names for per-topic metrics (bytes/messages in/out) is preventing these metrics from being reported to Yammer/Graphite. If this isn't intentional, it should probably be addressed before release. On Wed, Jan 21, 2015 at 9:28 AM, Jun

Re: [DISCUSSION] Boot dependency in the new producer

2015-01-26 Thread Jay Kreps
Hey Guozhang, That line shouldn't cause any connections to Kafka to be established, does it? All that is doing is creating the Cluster pojo using the supplied addresses. The use of InetSocketAddress may cause some dns stuff to happen, though... -Jay On Mon, Jan 26, 2015 at 10:50 AM, Guozhang Wan

Re: What to do when file.rename fails?

2015-01-26 Thread Jay Kreps
Also, I think I agree that shutting down is the right behavior. I think the real thing to do though is probably to debug that user's case and figure out if it is something inside kafka that is leading to double delete or if the files are getting deleted by something else they don't know about. Ther

Re: What to do when file.rename fails?

2015-01-26 Thread Jay Kreps
Having a relative path and keeping data under /data in the kafka distro would make sense. This would require some reworking of the shell scripts, though, as I think right now you an actually run Kafka from any directory and the cwd of the process will be whatever directory you start from. If we hav

[jira] [Commented] (KAFKA-1792) change behavior of --generate to produce assignment config with fair replica distribution and minimal number of reassignments

2015-01-26 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292368#comment-14292368 ] Joe Stein commented on KAFKA-1792: -- [~Dmitry Pekar] can you writeup a design for what you

[jira] [Comment Edited] (KAFKA-1792) change behavior of --generate to produce assignment config with fair replica distribution and minimal number of reassignments

2015-01-26 Thread Dmitry Pekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292355#comment-14292355 ] Dmitry Pekar edited comment on KAFKA-1792 at 1/26/15 8:27 PM: --

[jira] [Comment Edited] (KAFKA-1792) change behavior of --generate to produce assignment config with fair replica distribution and minimal number of reassignments

2015-01-26 Thread Dmitry Pekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292355#comment-14292355 ] Dmitry Pekar edited comment on KAFKA-1792 at 1/26/15 8:26 PM: --

[jira] [Commented] (KAFKA-1792) change behavior of --generate to produce assignment config with fair replica distribution and minimal number of reassignments

2015-01-26 Thread Dmitry Pekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292355#comment-14292355 ] Dmitry Pekar commented on KAFKA-1792: - [~nehanarkhede] thanks for your comments. Pleas

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Bhavesh Mistry
Hi Kafka Team, I just wanted to bring this to your attention regarding Java New Producer limitation compare to old producer. a) Partition Increasing is limited to configured memory allocation. buffer.memory batch.size The maximum partition you could have before impacting (New Java Producers)

[jira] [Commented] (KAFKA-1507) Using GetOffsetShell against non-existent topic creates the topic unintentionally

2015-01-26 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1507?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292220#comment-14292220 ] Jason Rosenberg commented on KAFKA-1507: I think relegating topic creation to an a

[DISCUSSION] Boot dependency in the new producer

2015-01-26 Thread Guozhang Wang
Hi all, I am not sure if we have discussed about this before, but recently I realized that we have introduced boot dependency of the kafka-server specified by the "bootstrap.servers" config in the new producer. More specifically, although in the old producer we also have a similar config for speci

Re: What to do when file.rename fails?

2015-01-26 Thread Guozhang Wang
Agree with Sriram / Jun, I think the error should be treated as fatal and we should shutdown the broker gracefully. On Mon, Jan 26, 2015 at 8:41 AM, Jun Rao wrote: > We probably can default the log dir to a relative path, sth like > ../kafka-logs. > > As for I/O errors on rename, I agree that we

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-01-26 Thread Jiangjie Qin
Hi Jay and Neha, Thanks a lot for the reply and explanation. I do agree it makes more sense to avoid duplicate effort and plan based on new consumer. I’ll modify the KIP. To Jay’s question on message ordering - The data channel selection makes sure that the messages from the same source partition

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Joe Stein
+1 (binding) artifacts and quick start look good. I ran in some client code, minor edits from 0-8.2-beta https://github.com/stealthly/scala-kafka/pull/26 On Mon, Jan 26, 2015 at 3:38 AM, Manikumar Reddy wrote: > +1 (Non-binding) > Verified source package, unit tests, release build, topic deleti

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

2015-01-26 Thread Navina Ramesh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292169#comment-14292169 ] Navina Ramesh commented on KAFKA-1861: -- [~nehanarkhede] I don't see it maven central.

Re: Review Request 30196: Patch for KAFKA-1886

2015-01-26 Thread Aditya Auradkar
> On Jan. 26, 2015, 1:28 a.m., Neha Narkhede wrote: > > core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala, line 235 > > > > > > what is the purpose of this sleep? I wanted to make sure the SimpleConsum

Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Jun Rao
+1 (binding) Verified quick start and unit tests. Thanks, Jun On Wed, Jan 21, 2015 at 8:28 AM, Jun Rao wrote: > This is the second candidate for release of Apache Kafka 0.8.2.0. There > has been some changes since the 0.8.2 beta release, especially in the new > java producer api and jmx mbean

[jira] [Updated] (KAFKA-1888) Add a "rolling upgrade" system test

2015-01-26 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1888: Assignee: (was: Gwen Shapira) > Add a "rolling upgrade" system test > --

[jira] [Commented] (KAFKA-1888) Add a "rolling upgrade" system test

2015-01-26 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1888?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292116#comment-14292116 ] Gwen Shapira commented on KAFKA-1888: - Since I'm not actively working on this patch an

Re: What to do when file.rename fails?

2015-01-26 Thread Jun Rao
We probably can default the log dir to a relative path, sth like ../kafka-logs. As for I/O errors on rename, I agree that we probably should just shut down the broker since it's not expected to happen. Thanks, Jun On Mon, Jan 26, 2015 at 5:54 AM, Jaikiran Pai wrote: > Having looked at the log

[jira] [Commented] (KAFKA-1889) Refactor shell wrapper scripts

2015-01-26 Thread Francois Saint-Jacques (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1889?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14291967#comment-14291967 ] Francois Saint-Jacques commented on KAFKA-1889: --- Do you think it could make

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-01-26 Thread Jaikiran Pai
The main culprit is this thread which goes into "forever retry connection to a closed zookeeper" when I shutdown Kafka (via a Ctrl + C) after zookeeper has already been shutdown. I have attached the complete thread dump, but I don't know if it will be delivered to the mailing list. "Thread-2"

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

2015-01-26 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14291861#comment-14291861 ] Manikumar Reddy commented on KAFKA-1861: We already have kafka-client-test jar. Th

Re: Review Request 29755: Patch for KAFKA-1853

2015-01-26 Thread Jaikiran Pai
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29755/ --- (Updated Jan. 26, 2015, 2:11 p.m.) Review request for kafka. Bugs: KAFKA-1853

[jira] [Commented] (KAFKA-1853) Unsuccessful suffix rename of expired LogSegment can leak open files and also leave the LogSegment in an invalid state

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

[jira] [Updated] (KAFKA-1853) Unsuccessful suffix rename of expired LogSegment can leak open files and also leave the LogSegment in an invalid state

2015-01-26 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1853: Attachment: KAFKA-1853_2015-01-26_19:41:33.patch > Unsuccessful suffix rename of expired LogSegment

Re: What to do when file.rename fails?

2015-01-26 Thread Jaikiran Pai
Having looked at the logs the user posted, I don't think this specific issue has to do with /tmp path. However, now that the /tmp path is being discussed, I think it's a good idea that we default the Kafka logs to a certain folder. As Jay notes, it makes it very easy to just download and start

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 2 (with the correct links)

2015-01-26 Thread Manikumar Reddy
+1 (Non-binding) Verified source package, unit tests, release build, topic deletion, compaction and random testing On Mon, Jan 26, 2015 at 6:14 AM, Neha Narkhede wrote: > +1 (binding) > Verified keys, quick start, unit tests. > > On Sat, Jan 24, 2015 at 4:26 PM, Joe Stein wrote: > > > That make