Re: New Consumer API and Range Consumption with Fail-over

2015-08-04 Thread Bhavesh Mistry
Hi Jason and Kafka Dev Team, First of all thanks for responding and I think you got expected behavior correctly. The use-case is offset range consumption. We store each minute highest offset for each topic per partition. So if we need to reload or re-consume data from yesterday per say 8AM

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Ben Kirwin
This is a great summary of the commit log options. Some additional comments: 1. One way to handle a transient failure is to treat it the same way as a conditional publish failure: recompute the post-value before retrying. (I believe this is enough to make this approach work correctly.) 2. You can

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Ben Kirwin
Jiangjie -- it seems to me that, while there are cases where you want conservative producer settings like you suggest, there are others enabled by this KIP where pipelining and retries are not an issue. As a toy example, I've adapted the producer performance test to behave as an idempotent produce

[GitHub] kafka pull request: KAFKA-2384; Encode/decode to utf-8 for commit ...

2015-08-04 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/109 KAFKA-2384; Encode/decode to utf-8 for commit title IO in kafka-merge-pr.py This fix should be fine for Linux and OS X. Not sure about Windows though. This is a very specific fix for new functionality

[jira] [Commented] (KAFKA-2384) Override commit message title in kafka-merge-pr.py

2015-08-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2384?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653344#comment-14653344 ] ASF GitHub Bot commented on KAFKA-2384: --- GitHub user ijuma opened a pull request:

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

2015-08-04 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: --- Status: Patch Available (was: Reopened) See PR #109 > Override commit message title in kafka-merge-p

[jira] [Commented] (KAFKA-1684) Implement TLS/SSL authentication

2015-08-04 Thread Navjot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653423#comment-14653423 ] Navjot commented on KAFKA-1684: --- I want to use this patch for my current project, please sug

[jira] [Commented] (KAFKA-1684) Implement TLS/SSL authentication

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653714#comment-14653714 ] Jun Rao commented on KAFKA-1684: [~NavjotBhardwaj], the ssl work is being covered in KAFKA

[jira] [Updated] (KAFKA-1684) Implement TLS/SSL authentication

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1684: --- Resolution: Duplicate Status: Resolved (was: Patch Available) The actual SSL work is now in KAFKA-169

[jira] [Comment Edited] (KAFKA-2285) Logging trait obfuscates call site information

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14652767#comment-14652767 ] Ismael Juma edited comment on KAFKA-2285 at 8/4/15 4:09 PM: Al

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

2015-08-04 Thread Jun Rao
A few questions/comments. 1. What's streamTime passed to punctuate()? Is that just the current time? 2. Is punctuate() only called if schedule() is called? 3. The way the KeyValueStore is created seems a bit weird. Since this is part of the internal state managed by KafkaProcessorContext, it seems

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Jay Kreps
Hey Jun, Yeah I think Ben is right, both these cases are covered. The algorithm is something like while(true) { v = get_local_val(key) v' = modify(v) try { log_to_kafka(v') put_local_val(k, v') break } catch(CasFailureException e) { warn("optimistic lock failure) } } W

[jira] [Commented] (KAFKA-2285) Logging trait obfuscates call site information

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653911#comment-14653911 ] Ismael Juma commented on KAFKA-2285: [~granthenke], keep in mind that scala-logging on

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Jun Rao
Jay, The code that you put there seems to be designed for the second case where there is a master. In the first case when there is no master, updates can happen from multiple replicas. Therefore, to maintain the local view, each replica can't just update the local view using only the updates sent

[jira] [Commented] (KAFKA-2285) Logging trait obfuscates call site information

2015-08-04 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653949#comment-14653949 ] Grant Henke commented on KAFKA-2285: Thanks for all the related Jiras and input [~ijum

[jira] [Commented] (KAFKA-2285) Logging trait obfuscates call site information

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14653951#comment-14653951 ] Ismael Juma commented on KAFKA-2285: Yes, agreed. > Logging trait obfuscates call sit

[jira] [Created] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-2403: -- Summary: Expose offset commit metadata in new consumer Key: KAFKA-2403 URL: https://issues.apache.org/jira/browse/KAFKA-2403 Project: Kafka Issue Type: I

[jira] [Commented] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654002#comment-14654002 ] Ismael Juma commented on KAFKA-2015: [~guozhang], is there a reason why this hasn't be

[jira] [Assigned] (KAFKA-1464) Add a throttling option to the Kafka replication tool

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-1464: -- Assignee: Ismael Juma (was: Neha Narkhede) > Add a throttling option to the Kafka replication

[jira] [Commented] (KAFKA-1464) Add a throttling option to the Kafka replication tool

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654015#comment-14654015 ] Ismael Juma commented on KAFKA-1464: I'd like to take a look at this. In a separate co

[jira] [Commented] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654035#comment-14654035 ] Guozhang Wang commented on KAFKA-2015: -- The topic subscription was not fully implemen

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Yasuhiro Matsuda
Jun, You are right that Jay's example omits the consuming part, and the local state should be maintained only by consumed messages so that all replicas are eventually consistent. Assuming that, the beauty of conditional push is that it is safe for clients to do this independently. It guarantees th

[jira] [Updated] (KAFKA-2015) Enable ConsoleConsumer to use new consumer

2015-08-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2015: --- Status: In Progress (was: Patch Available) OK, changing the status then. > Enable ConsoleConsumer to

[jira] [Commented] (KAFKA-2400) Expose heartbeat frequency in new consumer configuration

2015-08-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654050#comment-14654050 ] Jason Gustafson commented on KAFKA-2400: [~jkreps] I was thinking defaults in the

[jira] [Commented] (KAFKA-1464) Add a throttling option to the Kafka replication tool

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654052#comment-14654052 ] Jun Rao commented on KAFKA-1464: Another thing that we need to be a bit careful is that ty

[GitHub] kafka pull request: KAFKA-2055; Fix transient ConsumerBounceTest.t...

2015-08-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/98 --- 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 enable

[jira] [Commented] (KAFKA-2055) ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failure

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

[jira] [Resolved] (KAFKA-2055) ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failure

2015-08-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2055. -- Resolution: Fixed Fix Version/s: 0.8.3 Issue resolved by pull request 98 [https://github.

[jira] [Updated] (KAFKA-2402) Broker should create zkpath /isr_change_notification if it does not exist when updating ISR.

2015-08-04 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2402: Status: Patch Available (was: Open) > Broker should create zkpath /isr_change_notification if it do

[GitHub] kafka pull request: KAFKA-2384; Encode/decode to utf-8 for commit ...

2015-08-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/109 --- 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-2384) Override commit message title in kafka-merge-pr.py

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

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-04 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654133#comment-14654133 ] Ashish K Singh commented on KAFKA-1690: --- [~harsha_ch] great work! It will be great t

Re: Kafka KIP discussion tomorrow

2015-08-04 Thread Jun Rao
Hi, Due to issues with goto meeting, we only had a short meeting today. The following is a summary of today's meeting notes. KIP-28: We discussed whether the KeyValueStore is optional or pluggable in processor. Guozhang will update the wiki on this. KIP-27: To determine how useful the conditiona

[jira] [Commented] (KAFKA-2402) Broker should create zkpath /isr_change_notification if it does not exist.

2015-08-04 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654145#comment-14654145 ] Ashish K Singh commented on KAFKA-2402: --- [~becket_qin] the patch looks good. However

[jira] [Updated] (KAFKA-2402) Broker should create zkpath /isr_change_notification if it does not exist.

2015-08-04 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2402: Summary: Broker should create zkpath /isr_change_notification if it does not exist. (was: Broker sh

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-04 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654150#comment-14654150 ] Ashish K Singh commented on KAFKA-1690: --- Yes, I did. Did you try with multiple broke

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-04 Thread Rajasekar Elango (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654142#comment-14654142 ] Rajasekar Elango commented on KAFKA-1690: - [~singhashish] I did not get that excep

[GitHub] kafka pull request: KAFKA-2071: Replace Producer Request/Response ...

2015-08-04 Thread dajac
GitHub user dajac opened a pull request: https://github.com/apache/kafka/pull/110 KAFKA-2071: Replace Producer Request/Response with their org.apache.kafka.common.requests equivalents This PR replaces all producer request/response with their common equivalents but doesn't touch old

[jira] [Commented] (KAFKA-2071) Replace Produce Request/Response with their org.apache.kafka.common.requests equivalents

2015-08-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654185#comment-14654185 ] ASF GitHub Bot commented on KAFKA-2071: --- GitHub user dajac opened a pull request:

[jira] [Updated] (KAFKA-2071) Replace Produce Request/Response with their org.apache.kafka.common.requests equivalents

2015-08-04 Thread David Jacot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot updated KAFKA-2071: --- Status: Patch Available (was: Open) > Replace Produce Request/Response with their org.apache.kafka.co

Re: Kafka KIP discussion tomorrow

2015-08-04 Thread Edward Ribeiro
​The tech lead at my former company did A LOT of research and experimentation with various online meetings services/softwares (join.me, G hangouts, goto, etc). Unfortunately, the most *usable* are goto meetings and Google Hangouts. Even so, many people have problem with one of the two. :( During t

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-04 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654233#comment-14654233 ] Sriharsha Chintalapani commented on KAFKA-1690: --- [~singhashish] sorry I am a

Re: Kafka KIP discussion tomorrow

2015-08-04 Thread Jun Rao
Hi, Edward, Thanks for the info. We do plan to have those meeting sessions recorded and available down the road. Until we figure that out, I will send out the meeting notes after each session. Jun On Tue, Aug 4, 2015 at 12:35 PM, Edward Ribeiro wrote: > ​The tech lead at my former company did

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654313#comment-14654313 ] Jun Rao commented on KAFKA-1690: [~erajasekar], another thing is that currently, we can ov

[jira] [Commented] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654318#comment-14654318 ] Jiangjie Qin commented on KAFKA-2403: - Can we put this as a sub task of KAFKA-2387, ju

[jira] [Assigned] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-2403: -- Assignee: Jason Gustafson > Expose offset commit metadata in new consumer > ---

[jira] [Updated] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2403: --- Issue Type: Sub-task (was: Improvement) Parent: KAFKA-2387 > Expose offset commit met

[jira] [Commented] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654323#comment-14654323 ] Jason Gustafson commented on KAFKA-2403: [~becket_qin] Done. I went ahead and assi

[jira] [Updated] (KAFKA-2400) Expose heartbeat frequency in new consumer configuration

2015-08-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2400?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2400: --- Issue Type: Sub-task (was: Improvement) Parent: KAFKA-2387 > Expose heartbeat frequen

Modified the contribution pages in the wiki

2015-08-04 Thread Gwen Shapira
Just FYI: I heavily refactored the Patch Review wiki page ( https://cwiki.apache.org/confluence/display/KAFKA/Patch+Review+Tool). I removed everything that duplicated (or contradicted) the contributor page (https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes), so there is

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-08-04 Thread Jiangjie Qin
Jun, I see. So this only applies to uncompressed messages. Maybe that is fine given most user will probably turn on compression? I think the first approach is a more general approach but from application point of view might harder to implement. I am thinking is it easier for the application simply

[jira] [Commented] (KAFKA-1683) Implement a "session" concept in the socket server

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654435#comment-14654435 ] Gwen Shapira commented on KAFKA-1683: - I started looking at this again, and I think th

[jira] [Commented] (KAFKA-2402) Broker should create zkpath /isr_change_notification if it does not exist.

2015-08-04 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654446#comment-14654446 ] Jiangjie Qin commented on KAFKA-2402: - Hi [~ashishujjain], I replied to your comments

[jira] [Commented] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654462#comment-14654462 ] Jun Rao commented on KAFKA-2205: [~aauradkar], the latest patch looks good overall. I hit

[jira] [Commented] (KAFKA-1367) Broker topic metadata not kept in sync with ZooKeeper

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654469#comment-14654469 ] Jun Rao commented on KAFKA-1367: [~singhashish], forgot to mention this earlier. Could you

[jira] [Created] (KAFKA-2404) Delete config znode when config values are empty

2015-08-04 Thread Aditya Auradkar (JIRA)
Aditya Auradkar created KAFKA-2404: -- Summary: Delete config znode when config values are empty Key: KAFKA-2404 URL: https://issues.apache.org/jira/browse/KAFKA-2404 Project: Kafka Issue Type

[jira] [Created] (KAFKA-2405) KafkaHealthCheck kills the JVM in handleSessionEstablishmentError

2015-08-04 Thread jaikiran pai (JIRA)
jaikiran pai created KAFKA-2405: --- Summary: KafkaHealthCheck kills the JVM in handleSessionEstablishmentError Key: KAFKA-2405 URL: https://issues.apache.org/jira/browse/KAFKA-2405 Project: Kafka

[jira] [Commented] (KAFKA-2403) Expose offset commit metadata in new consumer

2015-08-04 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654479#comment-14654479 ] Jiangjie Qin commented on KAFKA-2403: - Sure, thanks! > Expose offset commit metadata

[GitHub] kafka pull request: KAFKA-2405 Don't kill the JVM on session estab...

2015-08-04 Thread jaikiran
GitHub user jaikiran opened a pull request: https://github.com/apache/kafka/pull/111 KAFKA-2405 Don't kill the JVM on session establishment failure As noted in the JIRA https://issues.apache.org/jira/browse/KAFKA-2405 currently the KafkaHealthCheck causes the JVM to terminate in cas

[jira] [Commented] (KAFKA-2405) KafkaHealthCheck kills the JVM in handleSessionEstablishmentError

2015-08-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654481#comment-14654481 ] ASF GitHub Bot commented on KAFKA-2405: --- GitHub user jaikiran opened a pull request:

[jira] [Updated] (KAFKA-2405) KafkaHealthCheck kills the JVM in handleSessionEstablishmentError

2015-08-04 Thread jaikiran pai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-2405: Status: Patch Available (was: Open) Pull request sent https://github.com/apache/kafka/pull/111 >

Re: Modified the contribution pages in the wiki

2015-08-04 Thread Jiangjie Qin
Thanks Gwen, it is much cleaner now :) Just wondering do we have any plan of deprecating the patch review tool? Jiangjie (Becket) Qin On Tue, Aug 4, 2015 at 2:14 PM, Gwen Shapira wrote: > Just FYI: > > I heavily refactored the Patch Review wiki page ( > https://cwiki.apache.org/confluence/displ

Re: Review Request 35677: Patch for KAFKA-2288

2015-08-04 Thread Gwen Shapira
> On July 7, 2015, 5:22 p.m., Jun Rao wrote: > > Thanks for the patch. +1. You can commit after addressing the following > > minor comments. I'm fixing these, but there was also some rebasing, so another look at the patch for sanity will be appreciated :) > On July 7, 2015, 5:22 p.m., Jun Ra

[jira] [Commented] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-08-04 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654505#comment-14654505 ] Aditya Auradkar commented on KAFKA-2205: Thanks Jun. 1. Filed https://issues.apac

Re: Modified the contribution pages in the wiki

2015-08-04 Thread Gwen Shapira
There's always a plan! The contributor page only lists github as a valid contribution method. Theoretically the committers / reviewers should start asking contributors who upload patches to send PRs instead and point them at the contributors page ( https://cwiki.apache.org/confluence/display/KAFK

[jira] [Updated] (KAFKA-2288) Follow-up to KAFKA-2249 - reduce logging and testing

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2288: Attachment: KAFKA-2288_2015-08-04_16:19:37.patch > Follow-up to KAFKA-2249 - reduce logging and test

[jira] [Commented] (KAFKA-2288) Follow-up to KAFKA-2249 - reduce logging and testing

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

Re: Review Request 35677: Patch for KAFKA-2288

2015-08-04 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/35677/ --- (Updated Aug. 4, 2015, 11:19 p.m.) Review request for kafka. Bugs: KAFKA-2288

[GitHub] kafka pull request: KAFKA-2405 Don't kill the JVM on session estab...

2015-08-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/111 --- 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-2405) KafkaHealthCheck kills the JVM in handleSessionEstablishmentError

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

[jira] [Updated] (KAFKA-2405) KafkaHealthCheck kills the JVM in handleSessionEstablishmentError

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2405: Resolution: Fixed Reviewer: Gwen Shapira Fix Version/s: 0.8.3 Status: Reso

[jira] [Commented] (KAFKA-2071) Replace Produce Request/Response with their org.apache.kafka.common.requests equivalents

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654572#comment-14654572 ] Gwen Shapira commented on KAFKA-2071: - The change in the PR is not the intent of this

[jira] [Updated] (KAFKA-2071) Replace Produce Request/Response with their org.apache.kafka.common.requests equivalents

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2071: Reviewer: Gwen Shapira Status: In Progress (was: Patch Available) Cancelling patch since curr

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
> On Aug. 4, 2015, 3:28 a.m., Edward Ribeiro wrote: > > clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java, > > line 27 > > > > > > It's a very good pratice to make any field ``final``

[GitHub] kafka pull request: KAFKA-2340; improve KafkaConsumer Fetcher test...

2015-08-04 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/112 KAFKA-2340; improve KafkaConsumer Fetcher test coverage You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-2340 Alternative

[jira] [Commented] (KAFKA-2340) Add additional unit tests for new consumer Fetcher

2015-08-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654596#comment-14654596 ] ASF GitHub Bot commented on KAFKA-2340: --- GitHub user hachikuji opened a pull request

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
> On Aug. 4, 2015, 12:36 a.m., Joel Koshy wrote: > > core/src/main/scala/kafka/server/ClientQuotaMetrics.scala, line 98 > > > > > > A number of places in this patch use a : b instead of a: b. Highly > > stylistic, I'm

Re: Modified the contribution pages in the wiki

2015-08-04 Thread Jiangjie Qin
Yeah, sounds good! Maybe we can say that for new patches reviews, please use PR process in the contributing wiki as well? Or it is somewhere I missed it. Jiangjie (Becket) Qin On Tue, Aug 4, 2015 at 3:56 PM, Gwen Shapira wrote: > There's always a plan! > > The contributor page only lists gi

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

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2384: Resolution: Fixed Status: Resolved (was: Patch Available) Looks like this was committed but

[jira] [Commented] (KAFKA-2401) Fix transient failure of ProducerSendTest.testCloseWithZeroTimeoutFromSenderThread()

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

[GitHub] kafka pull request: KAFKA-2401: fix transient failure in ProducerS...

2015-08-04 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/113 KAFKA-2401: fix transient failure in ProducerSendTest.testCloseWithZeroTimeoutFromSenderThread You can merge this pull request into a Git repository by running: $ git pull https://github.com

Re: Review Request 35677: Patch for KAFKA-2288

2015-08-04 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/35677/#review94164 --- Ship it! Looks good to me. +1. Just a minor comment below. core/s

Re: Review Request 35677: Patch for KAFKA-2288

2015-08-04 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/35677/#review94166 --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala (lines

[jira] [Commented] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-08-04 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654664#comment-14654664 ] Aditya A Auradkar commented on KAFKA-2084: -- Updated reviewboard https://reviews.a

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated Aug. 5, 2015, 1:50 a.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Updated] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-08-04 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2084: - Attachment: KAFKA-2084_2015-08-04_18:50:51.patch > byte rate metrics per client ID (produc

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated Aug. 5, 2015, 1:51 a.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Commented] (KAFKA-2384) Override commit message title in kafka-merge-pr.py

2015-08-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2384?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654671#comment-14654671 ] Guozhang Wang commented on KAFKA-2384: -- That is right - thanks [~gwenshap]. > Overri

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated Aug. 5, 2015, 2:07 a.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Commented] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-08-04 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654682#comment-14654682 ] Aditya A Auradkar commented on KAFKA-2084: -- Updated reviewboard https://reviews.a

[jira] [Updated] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-08-04 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2084: - Attachment: KAFKA-2084_2015-08-04_19:07:46.patch > byte rate metrics per client ID (produc

Re: Review Request 33049: Patch for KAFKA-2084

2015-08-04 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated Aug. 5, 2015, 2:08 a.m.) Review request for kafka, Joel Koshy and Jun

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

2015-08-04 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654691#comment-14654691 ] Mayuresh Gharat commented on KAFKA-2351: [~junrao] would you mind taking another l

[jira] [Updated] (KAFKA-2288) Follow-up to KAFKA-2249 - reduce logging and testing

2015-08-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2288: Resolution: Fixed Reviewer: Jun Rao Fix Version/s: 0.8.3 Status: Resolved

Contributor list

2015-08-04 Thread Sreepathi
Hello, Can someone add me to the list of contributors ? My Jira id: sree2k -- *-- *Sreepathi

Re: Contributor list

2015-08-04 Thread Jun Rao
Added. Thanks, Jun On Tue, Aug 4, 2015 at 8:11 PM, Sreepathi wrote: > Hello, > > Can someone add me to the list of contributors ? > > My Jira id: sree2k > > -- > *-- *Sreepathi >

Re: Review Request 33378: Patch for KAFKA-2136

2015-08-04 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33378/#review94167 --- Thanks for the patch. A few comments below. clients/src/main/java/

[jira] [Commented] (KAFKA-2071) Replace Produce Request/Response with their org.apache.kafka.common.requests equivalents

2015-08-04 Thread David Jacot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654799#comment-14654799 ] David Jacot commented on KAFKA-2071: I thought that the intent was to only replace the

[jira] [Commented] (KAFKA-1367) Broker topic metadata not kept in sync with ZooKeeper

2015-08-04 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14654801#comment-14654801 ] Ashish K Singh commented on KAFKA-1367: --- [~junrao] sure will update the page. > Bro

[jira] [Resolved] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-08-04 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-2205. Resolution: Fixed Fix Version/s: 0.8.3 Thanks. I will close this jira then. > Generalize TopicConfig

  1   2   >