Re: [KIP-DISCUSSION] KIP-7 Security - IP Filtering

2015-03-03 Thread Jeff Holoman
Hey Joel, good questions As a first thought, my experience with customers in large corporate environments probably has me somewhat jaded :). You know it really shouldn't take 3 weeks to get ports opened on a load balancer, but, that really does happen. Coordination across those teams also can and

[jira] [Updated] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1988: - Resolution: Fixed Status: Resolved (was: Patch Available) > org.apache.kafka.common.utils

[jira] [Commented] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346274#comment-14346274 ] Guozhang Wang commented on KAFKA-1988: -- Thanks [~tongli], +1 and committed to trunk.

[jira] [Commented] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346253#comment-14346253 ] Ashish K Singh commented on KAFKA-1994: --- Updated reviewboard https://reviews.apache.

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Ashish Singh
> On March 4, 2015, 1:59 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/utils/ZkUtils.scala, lines 817-826 > > > > > > We will need to remember the result of client.exist(). If > > isNamespaceChecked is true, we w

[jira] [Updated] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-1994: -- Attachment: KAFKA-1994_2015-03-03_18:19:45.patch > Evaluate performance effect of chroot check o

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31711/ --- (Updated March 4, 2015, 2:19 a.m.) Review request for kafka. Bugs: KAFKA-1994

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Jun Rao
> On March 4, 2015, 1:59 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/utils/ZkUtils.scala, lines 817-826 > > > > > > We will need to remember the result of client.exist(). If > > isNamespaceChecked is true, we w

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Ashish Singh
> On March 4, 2015, 1:59 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/utils/ZkUtils.scala, lines 817-826 > > > > > > We will need to remember the result of client.exist(). If > > isNamespaceChecked is true, we w

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31711/#review75132 --- core/src/main/scala/kafka/utils/ZkUtils.scala

Re: [KIP-DISCUSSION] KIP-7 Security - IP Filtering

2015-03-03 Thread Joel Koshy
The proposal itself looks reasonable, but I have a couple of questions as you made reference to "operators of the system; and network team" in your wiki. - Are spoofing attacks a concern even with this in place? If so, it would require some sort of internal ingress filtering which presumably n

[jira] [Updated] (KAFKA-1982) change kafka.examples.Producer to use the new java producer

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-1982: -- Attachment: KAFKA-1982_2015-03-03_17:50:57.patch > change kafka.examples.Producer to use the new

Re: Review Request 31369: Patch for KAFKA-1982

2015-03-03 Thread Ashish Singh
> On March 3, 2015, 5:42 a.m., Jun Rao wrote: > > Thanks for the review Jun! Addressed your concerns. - Ashish --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31369/#review74895 -

Re: Review Request 31369: Patch for KAFKA-1982

2015-03-03 Thread Ashish Singh
> On Feb. 27, 2015, 7:29 p.m., Gwen Shapira wrote: > > Thanks for the patch, Ashish. Its shaping up to be a very useful example. > > Two comments: > > > > 1. I think the ser/de should be part of the example and not in "common", > > I'm not sure integer ser/de is useful enough to be distributed

[jira] [Commented] (KAFKA-1982) change kafka.examples.Producer to use the new java producer

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1982?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346214#comment-14346214 ] Ashish K Singh commented on KAFKA-1982: --- Updated reviewboard https://reviews.apache.

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Jiangjie Qin
> On March 4, 2015, midnight, Guozhang Wang wrote: > > core/src/main/scala/kafka/tools/MirrorMaker.scala, line 67 > > > > > > Are we sure that one producer io thread is sufficient for all cases? > > Jiangjie Qin wrote:

Re: Review Request 31369: Patch for KAFKA-1982

2015-03-03 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31369/ --- (Updated March 4, 2015, 1:51 a.m.) Review request for kafka. Bugs: KAFKA-1982

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Jiangjie Qin
> On March 4, 2015, 12:48 a.m., Guozhang Wang wrote: > > core/src/main/scala/kafka/tools/MirrorMaker.scala, line 524 > > > > > > On the other hand we should avoid checking for each message as > > System.currentTimeMil

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

2015-03-03 Thread Apache Jenkins Server
See

Re: Review Request 31591: Patch for KAFKA-1992

2015-03-03 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31591/#review75120 --- Thanks for the patch. A few minor comments below. core/src/main/sc

Jenkins build is back to normal : KafkaPreCommit #29

2015-03-03 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

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

[jira] [Updated] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

2015-03-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1992: Attachment: KAFKA-1992_2015-03-03_17:17:43.patch > Following KAFKA-1697, checkEnoughReplicasReachOff

Re: Review Request 31591: Patch for KAFKA-1992

2015-03-03 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31591/ --- (Updated March 4, 2015, 1:17 a.m.) Review request for kafka. Bugs: KAFKA-1992

Re: [VOTE] KIP-7 Security - IP Filtering

2015-03-03 Thread Jeff Holoman
Guozhang, The way the patch is implemented, the check is done in the acceptor thread accept() method of the Socket Server, just before connectionQuotas. Thanks Jeff On Tue, Mar 3, 2015 at 7:59 PM, Guozhang Wang wrote: > Jeff, > > I am wondering if the IP filtering rule can be enforced at the

[jira] [Commented] (KAFKA-902) Randomize backoff on the clients for metadata requests

2015-03-03 Thread Geoffrey Anderson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346149#comment-14346149 ] Geoffrey Anderson commented on KAFKA-902: - Created reviewboard https://reviews.apac

[jira] [Updated] (KAFKA-902) Randomize backoff on the clients for metadata requests

2015-03-03 Thread Geoffrey Anderson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Geoffrey Anderson updated KAFKA-902: Attachment: KAFKA-902.patch > Randomize backoff on the clients for metadata requests > --

Review Request 31715: Patch for KAFKA-902

2015-03-03 Thread Geoffrey Anderson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31715/ --- Review request for kafka. Bugs: KAFKA-902 https://issues.apache.org/jira/br

Re: Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31711/#review75115 --- Thanks for the patch. A comment below. core/src/main/scala/kafka/u

Re: [VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Guozhang Wang
+1 binding. On Tue, Mar 3, 2015 at 1:08 PM, Harsha wrote: > +1 non-binding > > On Tue, Mar 3, 2015, at 12:39 PM, Jeff Holoman wrote: > > +1 non-binding of course > > > > On Tue, Mar 3, 2015 at 3:18 PM, Joe Stein wrote: > > > > > +1 > > > > > > ~ Joe Stein > > > - - - - - - - - - - - - - - - - -

Re: [VOTE] KIP-7 Security - IP Filtering

2015-03-03 Thread Guozhang Wang
Jeff, I am wondering if the IP filtering rule can be enforced at the socket server level instead of the Kafka API level? Guozhang On Tue, Mar 3, 2015 at 2:24 PM, Jiangjie Qin wrote: > +1 (non-binding) > > On 3/3/15, 1:17 PM, "Gwen Shapira" wrote: > > >+1 (non-binding) > > > >On Tue, Mar 3, 20

[jira] [Resolved] (KAFKA-1976) transient unit test failure in ProducerFailureHandlingTest.testNotEnoughReplicasAfterBrokerShutdown

2015-03-03 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1976?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1976. Resolution: Duplicate Duplicate of KAFKA-1999. > transient unit test failure in > ProducerFailureHandlingT

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31306/#review75112 --- core/src/test/scala/unit/kafka/log/CleanerTest.scala

[jira] [Updated] (KAFKA-1999) Fix failing unit-test: kafka.api.ProducerFailureHandlingTest > testNotEnoughReplicasAfterBrokerShutdown

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

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/#review75110 --- core/src/main/scala/kafka/tools/MirrorMaker.scala

[jira] [Updated] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-1994: -- Attachment: KAFKA-1994.patch > Evaluate performance effect of chroot check on Topic creation > -

[jira] [Commented] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346125#comment-14346125 ] Ashish K Singh commented on KAFKA-1994: --- Created reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-1994: -- Status: Patch Available (was: Open) > Evaluate performance effect of chroot check on Topic crea

Review Request 31711: Patch for KAFKA-1994

2015-03-03 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31711/ --- Review request for kafka. Bugs: KAFKA-1994 https://issues.apache.org/jira/b

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Guozhang Wang
> On March 4, 2015, midnight, Guozhang Wang wrote: > > core/src/main/scala/kafka/tools/MirrorMaker.scala, line 67 > > > > > > Are we sure that one producer io thread is sufficient for all cases? > > Jiangjie Qin wrote:

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31306/#review75047 --- core/src/test/scala/unit/kafka/log/CleanerTest.scala

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Jiangjie Qin
> On March 4, 2015, midnight, Guozhang Wang wrote: > > core/src/main/scala/kafka/tools/MirrorMaker.scala, line 682 > > > > > > Instead of a wrapper rebalancer, I think it is cleaner to just > > instantiate the Consume

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346096#comment-14346096 ] Jiangjie Qin commented on KAFKA-1997: - Updated reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997_2015-03-03_16:28:46.patch > Refactor Mirror Maker > - > >

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/ --- (Updated March 4, 2015, 12:28 a.m.) Review request for kafka. Bugs: KAFKA-199

Build failed in Jenkins: Kafka-trunk #416

2015-03-03 Thread Apache Jenkins Server
See Changes: [jjkoshy] KAFKA-2001; Trivial commit to prevent OffsetCommitTest from hanging [jjkoshy] KAFKA-2001; Trivial commit to fix OffsetCommitTest -- [...truncated 1121 lines...] kafka.api.test.

[jira] [Commented] (KAFKA-902) Randomize backoff on the clients for metadata requests

2015-03-03 Thread Geoffrey Anderson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-902?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346083#comment-14346083 ] Geoffrey Anderson commented on KAFKA-902: - Thanks for the feedback. Is there any r

[jira] [Resolved] (KAFKA-2001) OffsetCommitTest hang during setup

2015-03-03 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy resolved KAFKA-2001. --- Resolution: Fixed This was due to a change I made as part of KAFKA-1755 I added compression.type "non

Re: Review Request 31566: Patch for KAFKA-1988

2015-03-03 Thread Tong Li
Guozhang, Yes, I have rebased and updated the patch set to resolve the couple comments for code comments. Also removed the trailing spaces (which always annoying. :-) ), Please see the new patch set here. Sorry for the delay. https://reviews.apache.org/r/31566/ Thanks. Tong Li OpenStack &

[jira] [Updated] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tong Li updated KAFKA-1988: --- Attachment: KAFKA-1988_2015-03-03_19:03:31.patch > org.apache.kafka.common.utils.Utils.abs method returns wron

[jira] [Commented] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346045#comment-14346045 ] Tong Li commented on KAFKA-1988: Updated reviewboard https://reviews.apache.org/r/31566/di

Re: Review Request 31566: Patch for KAFKA-1988

2015-03-03 Thread Tong Li
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31566/ --- (Updated March 4, 2015, 12:03 a.m.) Review request for kafka. Bugs: KAFKA-198

[jira] [Updated] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tong Li updated KAFKA-1988: --- Attachment: KAFKA-1988_2015-03-03_19:00:21.patch > org.apache.kafka.common.utils.Utils.abs method returns wron

[jira] [Commented] (KAFKA-1988) org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.

2015-03-03 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14346038#comment-14346038 ] Tong Li commented on KAFKA-1988: Updated reviewboard https://reviews.apache.org/r/31566/di

Re: Review Request 31566: Patch for KAFKA-1988

2015-03-03 Thread Tong Li
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31566/ --- (Updated March 4, 2015, midnight) Review request for kafka. Bugs: KAFKA-1988

Re: Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/#review75071 --- clients/src/main/java/org/apache/kafka/clients/producer/internals/R

[jira] [Assigned] (KAFKA-2001) OffsetCommitTest hang during setup

2015-03-03 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy reassigned KAFKA-2001: - Assignee: Joel Koshy I'll take a quick look at this > OffsetCommitTest hang during setup > -

Re: Review Request 31566: Patch for KAFKA-1988

2015-03-03 Thread Tong Li
> On March 3, 2015, 4:48 p.m., Guozhang Wang wrote: > > Tong, could you address Jun's last comments before committing? Yes. absolutely, doing it now. Thanks. - Tong --- This is an automatically generated e-mail. To reply, visit: https:/

Re: TL;DR; of the first KIP hangout

2015-03-03 Thread Joel Koshy
Thanks Gwen! On Tue, Mar 03, 2015 at 12:12:24PM -0800, Gwen Shapira wrote: > Hi, > > I put together a (very) short summary of the discussion and decisions: > > KIPs: > We reviewed the list of KIPs posted here: > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > > *

[jira] [Created] (KAFKA-2001) OffsetCommitTest hang during setup

2015-03-03 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2001: -- Summary: OffsetCommitTest hang during setup Key: KAFKA-2001 URL: https://issues.apache.org/jira/browse/KAFKA-2001 Project: Kafka Issue Type: Bug Components: co

Re: TL;DR; of the first KIP hangout

2015-03-03 Thread Jay Kreps
Thanks Gwen! -Jay On Tue, Mar 3, 2015 at 12:12 PM, Gwen Shapira wrote: > Hi, > > I put together a (very) short summary of the discussion and decisions: > > KIPs: > We reviewed the list of KIPs posted here: > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > > * K

Build failed in Jenkins: KafkaPreCommit #28

2015-03-03 Thread Apache Jenkins Server
See Changes: [jjkoshy] KAFKA-1755; Reject compressed and unkeyed messages sent to compacted topics; reviewed by Mayuresh Gharat, Neha Narkhede and Guozhang Wang [jjkoshy] KAFKA-1852; Reject offset commits to unknown topics; reviewed by J

[jira] [Commented] (KAFKA-1660) Ability to call close() with a timeout on the Java Kafka Producer.

2015-03-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14345918#comment-14345918 ] Guozhang Wang commented on KAFKA-1660: -- +1 on creating a new KIP. Especially, I think

[jira] [Commented] (KAFKA-1994) Evaluate performance effect of chroot check on Topic creation

2015-03-03 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14345891#comment-14345891 ] Ashish K Singh commented on KAFKA-1994: --- [~junrao] Your suggestion makes sense. Prof

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Attachment: KAFKA-1997.patch > Refactor Mirror Maker > - > > Key

[jira] [Updated] (KAFKA-1997) Refactor Mirror Maker

2015-03-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-1997: Status: Patch Available (was: Open) > Refactor Mirror Maker > - > >

[jira] [Commented] (KAFKA-1997) Refactor Mirror Maker

2015-03-03 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14345879#comment-14345879 ] Jiangjie Qin commented on KAFKA-1997: - Created reviewboard https://reviews.apache.org/

Re: [VOTE] KIP-7 Security - IP Filtering

2015-03-03 Thread Jiangjie Qin
+1 (non-binding) On 3/3/15, 1:17 PM, "Gwen Shapira" wrote: >+1 (non-binding) > >On Tue, Mar 3, 2015 at 12:44 PM, Jeff Holoman >wrote: >> Details in the wiki. >> >> >> >>https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+F >>iltering >> >> >> >> -- >> Jeff Holoman >> System

Review Request 31706: Patch for KAFKA-1997

2015-03-03 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31706/ --- Review request for kafka. Bugs: KAFKA-1997 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

2015-03-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1992: Attachment: KAFKA-1992_2015-03-03_14:16:34.patch > Following KAFKA-1697, checkEnoughReplicasReachOff

Re: Review Request 31591: Patch for KAFKA-1992

2015-03-03 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31591/ --- (Updated March 3, 2015, 10:16 p.m.) Review request for kafka. Bugs: KAFKA-199

[jira] [Commented] (KAFKA-1992) Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks

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

Build failed in Jenkins: Kafka-trunk #415

2015-03-03 Thread Apache Jenkins Server
See Changes: [jjkoshy] KAFKA-1755; Reject compressed and unkeyed messages sent to compacted topics; reviewed by Mayuresh Gharat, Neha Narkhede and Guozhang Wang [jjkoshy] KAFKA-1852; Reject offset commits to unknown topics; reviewed by Joe

Re: [VOTE] KIP-7 Security - IP Filtering

2015-03-03 Thread Gwen Shapira
+1 (non-binding) On Tue, Mar 3, 2015 at 12:44 PM, Jeff Holoman wrote: > Details in the wiki. > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+Filtering > > > > -- > Jeff Holoman > Systems Engineer

Re: [VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Harsha
+1 non-binding On Tue, Mar 3, 2015, at 12:39 PM, Jeff Holoman wrote: > +1 non-binding of course > > On Tue, Mar 3, 2015 at 3:18 PM, Joe Stein wrote: > > > +1 > > > > ~ Joe Stein > > - - - - - - - - - - - - - - - - - > > > > http://www.stealth.ly > > - - - - - - - - - - - - - - - - - > > > > O

Re: TL;DR; of the first KIP hangout

2015-03-03 Thread Guozhang Wang
Thanks Gwen! One thing that I forgot to mention during the meeting is that we need to create a KIP for another producer API change also (KAFKA-1660). The code is almost ready and we just need to figure out some details about interacting with concurrent flush() calls. Guozhang On Tue, Mar 3, 2015

[VOTE] KIP-7 Security - IP Filtering

2015-03-03 Thread Jeff Holoman
Details in the wiki. https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+Filtering -- Jeff Holoman Systems Engineer

Re: [VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Jeff Holoman
+1 non-binding of course On Tue, Mar 3, 2015 at 3:18 PM, Joe Stein wrote: > +1 > > ~ Joe Stein > - - - - - - - - - - - - - - - - - > > http://www.stealth.ly > - - - - - - - - - - - - - - - - - > > On Tue, Mar 3, 2015 at 3:14 PM, Gwen Shapira > wrote: > > > Details are in the wiki: > > > > > >

Re: [VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Joe Stein
+1 ~ Joe Stein - - - - - - - - - - - - - - - - - http://www.stealth.ly - - - - - - - - - - - - - - - - - On Tue, Mar 3, 2015 at 3:14 PM, Gwen Shapira wrote: > Details are in the wiki: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-2+-+Refactor+brokers+to+allow+listening+on+multi

[VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Gwen Shapira
Details are in the wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-2+-+Refactor+brokers+to+allow+listening+on+multiple+ports+and+IPs

TL;DR; of the first KIP hangout

2015-03-03 Thread Gwen Shapira
Hi, I put together a (very) short summary of the discussion and decisions: KIPs: We reviewed the list of KIPs posted here: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals * KIP-2: Ready for formal vote * KIP-3: Discussion closed. There’s a new KIP (14) for standardi

Re: [kafka-clients] Re: [VOTE] 0.8.2.1 Candidate 2

2015-03-03 Thread Joe Stein
Ok, lets fix the transient test failure on trunk agreed not a blocker. +1 quick start passed, verified artifacts, updates in scala https://github.com/stealthly/scala-kafka/tree/0.8.2.1 and go https://github.com/stealthly/go_kafka_client/tree/0.8.2.1 look good ~ Joe Stein - - - - - - - - - - - - -

[jira] [Created] (KAFKA-2000) Delete consumer offsets from kafka once the topic is deleted

2015-03-03 Thread Sriharsha Chintalapani (JIRA)
Sriharsha Chintalapani created KAFKA-2000: - Summary: Delete consumer offsets from kafka once the topic is deleted Key: KAFKA-2000 URL: https://issues.apache.org/jira/browse/KAFKA-2000 Project:

[jira] [Updated] (KAFKA-1986) Producer request failure rate should not include InvalidMessageSizeException and OffsetOutOfRangeException

2015-03-03 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1986: -- Resolution: Fixed Status: Resolved (was: Patch Available) Committed to trunk > Producer reques

[jira] [Updated] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic

2015-03-03 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1852: -- Resolution: Fixed Status: Resolved (was: Patch Available) committed to trunk > OffsetCommitReq

[jira] [Commented] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic

2015-03-03 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14345612#comment-14345612 ] Joel Koshy commented on KAFKA-1852: --- (As noted in RB we need a separate jira to handle f

Re: Review Request 29467: Patch for KAFKA-1660

2015-03-03 Thread Parth Brahmbhatt
> On March 3, 2015, 5:37 a.m., Jay Kreps wrote: > > clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, > > line 533 > > > > > > Now there is a bit of duplicate code between the two close metho

Re: Review Request 29467: Patch for KAFKA-1660

2015-03-03 Thread Parth Brahmbhatt
> On March 3, 2015, 4:10 a.m., Jay Kreps wrote: > > clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, > > line 560 > > > > > > This seems to call initiateClose() twice, once in initiateClose

Re: [VOTE] KIP-3 Mirror Maker Enhancement

2015-03-03 Thread Jiangjie Qin
KIP-3 passed with 3 binding +1 and 2 non-binding +1. On 2/26/15, 9:02 PM, "Neha Narkhede" wrote: >+1 (binding) > >On Thu, Feb 26, 2015 at 2:14 PM, Joel Koshy wrote: > >> +1 (binding) >> >> On Tue, Feb 24, 2015 at 06:37:53PM -0800, Guozhang Wang wrote: >> > +1 binding >> > >> > On Tue, Feb 24, 2

Re: [DISCUSS] KIP-5 - Broker Configuration Management

2015-03-03 Thread Andrii Biletskyi
Hey Jun, Initially I was thinking about instead of 3-4 state that global config changes take effect only after broker restart. So it's just: 3-4. On each broker startup apply global config from ZK In other words, the comprehensive workflow is the following: 1. Issue ChangeGlobalConfigRequest 2.

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-03-03 Thread Joel Koshy
Thanks for sending that out Joe - I don't think I will be able to make it today, so if notes can be sent out afterward that would be great. On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote: > Thanks for sending this out Joe. Looking forward to chatting with everyone :) > > On Mon, Mar

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31306/#review74992 --- Ship it! Ship It! - Mayuresh Gharat On Feb. 26, 2015, 6:54 p.m.,

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Joel Koshy
> On March 3, 2015, 5:43 p.m., Mayuresh Gharat wrote: > > core/src/main/scala/kafka/log/LogCleaner.scala, line 413 > > > > > > This will mean that if there are unkeyed messages we will neglect them > > and not throw a

[jira] [Resolved] (KAFKA-789) Producer-side persistence for delivery guarantee

2015-03-03 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-789. - Resolution: Duplicate > Producer-side persistence for delivery guarantee >

Re: [DISCUSS] KIP-5 - Broker Configuration Management

2015-03-03 Thread Jun Rao
It seems the proposed workflow is the following. 1. Client issues a global config update request to the broker. 2. Broker writes the new config to ZK. 3. Controller picks up the changes from ZK. 4. Controller propagates the config changes to all brokers. Do we need to add a new request/response t

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31306/#review74986 --- core/src/main/scala/kafka/log/LogCleaner.scala

Re: [kafka-clients] Re: [VOTE] 0.8.2.1 Candidate 2

2015-03-03 Thread Jun Rao
Hi, Joe, Yes, that unit test does have transient failures from time to time. The issue seems to be with the unit test itself and not the actual code. So, this is not a blocker for 0.8.2.1 release. I think we can just fix it in trunk. Thanks, Jun On Tue, Mar 3, 2015 at 9:08 AM, Joe Stein wrote:

Re: Review Request 31306: Patch for KAFKA-1755

2015-03-03 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31306/#review74985 --- Ship it! Ship It! - Guozhang Wang On Feb. 26, 2015, 6:54 p.m., J

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

2015-03-03 Thread Jun Rao
It would also be good to think through how we can use those new admin requests in the producer/consumer client as well. Currently, both the producer and the consumer use TopicMetadataRequest to obtain the metadata, which will trigger a topic creation if auto topic creation is enabled. This is a bit

Re: [kafka-clients] Re: [VOTE] 0.8.2.1 Candidate 2

2015-03-03 Thread Gwen Shapira
Hi, Good catch, Joe. Releasing with a broken test is not a good habit. I provided a small patch that fixes the issue in KAFKA-1999. Gwen On Tue, Mar 3, 2015 at 9:08 AM, Joe Stein wrote: > Jun, I have most everything looks good except I keep getting test failures > from wget > https://people.a

  1   2   >