Re: Review Request 33383: Patch for KAFKA-1595

2015-04-21 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33383/ --- (Updated April 21, 2015, 7:03 a.m.) Review request for kafka. Bugs: KAFKA-159

[jira] [Commented] (KAFKA-2024) Cleaner can generate unindexable log segments

2015-04-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14504654#comment-14504654 ] Rajini Sivaram commented on KAFKA-2024: --- [~mgharat] 32-bit relative offsets are stor

[jira] [Commented] (KAFKA-2036) Consumer and broker have different networks

2015-04-21 Thread Arsenii Krasikov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14504876#comment-14504876 ] Arsenii Krasikov commented on KAFKA-2036: - I find that {{InetSocketAddress}} is on

[jira] [Updated] (KAFKA-2036) Consumer and broker have different networks

2015-04-21 Thread Arsenii Krasikov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Arsenii Krasikov updated KAFKA-2036: Attachment: patch2 > Consumer and broker have different networks > -

[jira] [Updated] (KAFKA-2036) Consumer and broker have different networks

2015-04-21 Thread Arsenii Krasikov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Arsenii Krasikov updated KAFKA-2036: Attachment: patch2 > Consumer and broker have different networks > -

[jira] [Commented] (KAFKA-2036) Consumer and broker have different networks

2015-04-21 Thread Arsenii Krasikov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14504906#comment-14504906 ] Arsenii Krasikov commented on KAFKA-2036: - {{patch}} is tested in production, {{pa

[jira] [Updated] (KAFKA-2036) Consumer and broker have different networks

2015-04-21 Thread Arsenii Krasikov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Arsenii Krasikov updated KAFKA-2036: Attachment: (was: patch2) > Consumer and broker have different networks > --

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-04-21 Thread Andrii Biletskyi
Hi all, I've updated KIP-4 page to include all previously discussed items such as: new error codes, merged alter-topic and reassign-partitions requests, added TMR_V1. It'd be great if we concentrate on the Errors+Wire Protocol schema and discuss any remaining issues today, since first patch will

[jira] [Created] (KAFKA-2137) New Kafka Producer not fully asynchronous

2015-04-21 Thread David Hay (JIRA)
David Hay created KAFKA-2137: Summary: New Kafka Producer not fully asynchronous Key: KAFKA-2137 URL: https://issues.apache.org/jira/browse/KAFKA-2137 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Jun Rao
Harsha, Parth, Thanks for the clarification. This makes sense. Perhaps we can clarify the meaning of those rules in the wiki. Related to this, it seems that we need to support wildcard in cli/request protocol for topics? Jun On Mon, Apr 20, 2015 at 9:07 PM, Parth Brahmbhatt < pbrahmbh...@horton

[jira] [Resolved] (KAFKA-2137) New Kafka Producer not fully asynchronous

2015-04-21 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-2137. -- Resolution: Duplicate This is the same issue as KAFKA-1835, which has a patch bu

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-04-21 Thread Jay Kreps
I'm also +1 on this. The change is quite small and may actually help perf on Linux as well (we've never tried this). I have a lot of concerns on testing the various failure conditions but I think since it will be off by default the risk is not too high. -Jay On Mon, Apr 20, 2015 at 6:58 PM, Hong

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-04-21 Thread Neha Narkhede
+1. I've tried this on Linux and it helps reduce the spikes in append (and hence producer) latency for high throughput writes. I am not entirely sure why but my suspicion is that in the absence of preallocation, you see spikes writes need to happen faster than the time it takes Linux to allocate th

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-04-21 Thread David Hay (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505238#comment-14505238 ] David Hay commented on KAFKA-1835: -- This solution doesn't seem ideal to me. It requires

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-21 Thread Joel Koshy
Given the caveats, it may be worth doing further investigation on the alternate approach which is to use a dedicated DelayQueue for requests that violate quota and compare pros/cons. So the approach is the following: all request handling occurs normally (i.e., unchanged from what we do today). i.e

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505329#comment-14505329 ] Jay Kreps commented on KAFKA-2132: -- Yeah it definitely makes sense to move the log4j appe

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-21 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505370#comment-14505370 ] Joe Stein commented on KAFKA-2132: -- Can we put it in the new new admin client tools jar t

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-04-21 Thread Sriram Subramanian
Could you describe how recovery works in this mode? Say, we had a 250 MB preallocated segment and we wrote till 50MB and crashed. Till what point do we recover? Also, on startup, how is the append end pointer set even on a clean shutdown? How does the FileChannel end position get set to 50 MB inste

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505376#comment-14505376 ] Jay Kreps commented on KAFKA-2132: -- Shouldn't the admin stuff be part of the main client

Re: Review Request 33049: Patch for KAFKA-2084

2015-04-21 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated April 21, 2015, 7:21 p.m.) Review request for kafka and Joel Koshy.

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

2015-04-21 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505563#comment-14505563 ] 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-04-21 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-04-21_12:21:18.patch > byte rate metrics per client ID (produc

Re: Review Request 33049: Patch for KAFKA-2084

2015-04-21 Thread Aditya Auradkar
> On April 17, 2015, 11:21 p.m., Joel Koshy wrote: > > core/src/main/scala/kafka/server/ClientQuotaMetrics.scala, line 33 > > > > > > Is this necessary? Not strictly but I felt it was a nice to have. I can make the def

Re: Review Request 33049: Patch for KAFKA-2084

2015-04-21 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated April 21, 2015, 7:28 p.m.) Review request for kafka and Joel Koshy.

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

2015-04-21 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505574#comment-14505574 ] 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-04-21 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-04-21_12:28:05.patch > byte rate metrics per client ID (produc

Re: Review Request 33049: Patch for KAFKA-2084

2015-04-21 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated April 21, 2015, 7:33 p.m.) Review request for kafka and Joel Koshy.

[jira] [Commented] (KAFKA-2046) Delete topic still doesn't work

2015-04-21 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505688#comment-14505688 ] Sriharsha Chintalapani commented on KAFKA-2046: --- [~onurkaraman] I am curious

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-21 Thread Joel Koshy
In either approach I'm not sure we considered being able to turn it off completely. IOW, no it is not a "plugin" if that's what you are asking. We can set very high defaults by default and in the absence of any overrides it would effectively be off. The quota enforcement is actually already part of

Review Request 33410: Patch for KAFKA-2034

2015-04-21 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33410/ --- Review request for kafka. Bugs: KAFKA-2034 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-2034) sourceCompatibility not set in Kafka build.gradle

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2034?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2034: --- Attachment: KAFKA-2034.patch > sourceCompatibility not set in Kafka build.gradle > ---

[jira] [Updated] (KAFKA-2034) sourceCompatibility not set in Kafka build.gradle

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2034?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2034: --- Assignee: Ismael Juma Status: Patch Available (was: Open) > sourceCompatibility not set in Kafk

[jira] [Commented] (KAFKA-2034) sourceCompatibility not set in Kafka build.gradle

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14505742#comment-14505742 ] Ismael Juma commented on KAFKA-2034: Created reviewboard https://reviews.apache.org/r/

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Jay Kreps
Hey Parth, Great write-up! One super minor thing: could we change the "EDIT" permission to be called "ALTER"? The request name in KIP-4 is Alter and the command line tool has always been alter (or we could go the other way and change those to EDIT). Not sure that one is any better than the other

Re: Review Request 33410: Patch for KAFKA-2034

2015-04-21 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33410/ --- (Updated April 21, 2015, 9 p.m.) Review request for kafka. Bugs: KAFKA-2034

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Jay Kreps
Also, I think I may have missed this but does READ imply you also have DESCRIBE? A reader will need access to both read offsets (to determine their own initial position) as well as commit offsets. Currently, though fetching offsets is under DESCRIBE only and commit offsets is under READ. If READ=>D

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-04-21 Thread Jay Kreps
My understanding of the patch is that clean shutdown truncates the file back to it's true size (and reallocates it on startup). Hard crash is handled by the normal recovery which should truncate off the empty portion of the file. On Tue, Apr 21, 2015 at 10:52 AM, Sriram Subramanian < srsubraman...

[jira] [Created] (KAFKA-2138) KafkaProducer does not honor the retry backoff time.

2015-04-21 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2138: --- Summary: KafkaProducer does not honor the retry backoff time. Key: KAFKA-2138 URL: https://issues.apache.org/jira/browse/KAFKA-2138 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Parth Brahmbhatt
Changed Edit to Alter. I did not think about it that way but Sriharsha raised the same point in a private conversation. I did not think about it that way but I agree it makes sense. If no one objects I think in default implementation we can infer that if user have READ or WRITE access he gets DESC

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-04-21 Thread Jay Kreps
Hey Andrii, thanks for all the hard work on this, it has come a long way. A couple questions and comments on this. For the errors, can we do the following: 1. Remove IllegalArgument from the name, we haven't used that convention for other errors. 2. Normalize this list with the existing errors. F

Re: KIP hangout on Apr 21

2015-04-21 Thread Tong Li
Jun, Not sure why in these invitations, I am not seeing the google hangout link. I am using IBM notes which is quite different from gmail and calendar. Is there anyway that you send the hangout link in the invitation body? Thanks. Tong Li OpenStack & Kafka Community Development Building 501/

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Parth Brahmbhatt
Hey Jun, Yes and we support wild cards for all acl entities principal, hosts and operation. Thanks Parth On 4/21/15, 9:06 AM, "Jun Rao" wrote: >Harsha, Parth, > >Thanks for the clarification. This makes sense. Perhaps we can clarify the >meaning of those rules in the wiki. > >Related to this,

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-21 Thread Jay Kreps
Hey Sriharsha, Thanks for the excellent write-up. Couple of minor questions: 1. Isn't the blocking handshake going to be a performance concern? Can we do the handshake non-blocking instead? If anything that causes connections to drop can incur blocking network roundtrips won't that eat up all th

[jira] [Created] (KAFKA-2139) Add a separate controller messge queue with higher priority on broker side

2015-04-21 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2139: --- Summary: Add a separate controller messge queue with higher priority on broker side Key: KAFKA-2139 URL: https://issues.apache.org/jira/browse/KAFKA-2139 Project: Kafk

[jira] [Assigned] (KAFKA-2139) Add a separate controller messge queue with higher priority on broker side

2015-04-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin reassigned KAFKA-2139: --- Assignee: Jiangjie Qin > Add a separate controller messge queue with higher priority on broke

Review Request 33417: Patch for KAFKA-2138

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

[jira] [Updated] (KAFKA-2138) KafkaProducer does not honor the retry backoff time.

2015-04-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2138?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2138: Attachment: KAFKA-2138.patch > KafkaProducer does not honor the retry backoff time. > --

[jira] [Updated] (KAFKA-2138) KafkaProducer does not honor the retry backoff time.

2015-04-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2138?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2138: Status: Patch Available (was: Open) > KafkaProducer does not honor the retry backoff time. > --

[jira] [Commented] (KAFKA-2138) KafkaProducer does not honor the retry backoff time.

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

[jira] [Updated] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists

2015-04-21 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-1810: Resolution: Won't Fix Status: Resolved (was: Patch Available) > Add IP Filtering / Whitelis

[jira] [Commented] (KAFKA-2138) KafkaProducer does not honor the retry backoff time.

2015-04-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506011#comment-14506011 ] Jiangjie Qin commented on KAFKA-2138: - [~junrao] [~jjkoshy] Do you think we should als

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Jay Kreps
Following up on the KIP discussion. Two options for authorizing consumers to read topic "t" as part of group "g": 1. READ permission on resource /topic/t 2. READ permission on resource /topic/t AND WRITE permission on /group/g The advantage of (1) is that it is simpler. The disadvantage is that an

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Gwen Shapira
Adding my notes from today's call to the thread: ** Deny or Allow all by default? We will add a configuration to control this. The configuration will default to “allow” for backward compatibility. Security admins can set it to "deny" ** Storing ACLs for default authorizers: We'll store them in ZK

Re: Review Request 33417: Patch for KAFKA-2138

2015-04-21 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33417/#review81097 --- This piece of logic has been quite complex and awkward to me now, fo

[jira] [Created] (KAFKA-2140) Improve code readability

2015-04-21 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2140: -- Summary: Improve code readability Key: KAFKA-2140 URL: https://issues.apache.org/jira/browse/KAFKA-2140 Project: Kafka Issue Type: Improvement Report

Review Request 33420: Patch for KAFKA-2140

2015-04-21 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33420/ --- Review request for kafka. Bugs: KAFKA-2140 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-2140) Improve code readability

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506066#comment-14506066 ] Ismael Juma commented on KAFKA-2140: Created reviewboard https://reviews.apache.org/r/

[jira] [Created] (KAFKA-2141) Integrate checkstyle for Java code

2015-04-21 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2141: Summary: Integrate checkstyle for Java code Key: KAFKA-2141 URL: https://issues.apache.org/jira/browse/KAFKA-2141 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-2140) Improve code readability

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2140: --- Status: Patch Available (was: Open) > Improve code readability > > >

[jira] [Updated] (KAFKA-2140) Improve code readability

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2140: --- Attachment: KAFKA-2140.patch > Improve code readability > > >

[jira] [Commented] (KAFKA-2140) Improve code readability

2015-04-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506067#comment-14506067 ] Ismael Juma commented on KAFKA-2140: This PR shows the changes in an easy to review wa

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-04-21 Thread Andrii Biletskyi
Guys, Thank you for your time. A short summary of our discussion. Answering previous items: 1. 2. I will double check existing error codes to align the list of errors that needs to be added. 3. We agreed to think again about the batch requests semantics. The main concern is that users would expe

[jira] [Commented] (KAFKA-2029) Improving controlled shutdown for rolling updates

2015-04-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506078#comment-14506078 ] Jay Kreps commented on KAFKA-2029: -- I don't have a ton of context on this chunk of code.

Review Request 33421: Patch for KAFKA-2114

2015-04-21 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33421/ --- Review request for kafka. Bugs: KAFKA-2114 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-2114) Unable to change min.insync.replicas default

2015-04-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2114: Attachment: KAFKA-2114.patch > Unable to change min.insync.replicas default > --

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-21 Thread Parth Brahmbhatt
I have added the notes to KIP-11 Open question sections. Thanks Parth On 4/21/15, 4:49 PM, "Gwen Shapira" wrote: >Adding my notes from today's call to the thread: > >** Deny or Allow all by default? We will add a configuration to >control this. The configuration will default to “allow” for back

[jira] [Updated] (KAFKA-2114) Unable to change min.insync.replicas default

2015-04-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2114: Status: Patch Available (was: Open) > Unable to change min.insync.replicas default > --

[jira] [Commented] (KAFKA-2114) Unable to change min.insync.replicas default

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

[jira] [Commented] (KAFKA-2114) Unable to change min.insync.replicas default

2015-04-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506084#comment-14506084 ] Gwen Shapira commented on KAFKA-2114: - Ended up being very silly case of not passing t

[jira] [Commented] (KAFKA-2139) Add a separate controller messge queue with higher priority on broker side

2015-04-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506085#comment-14506085 ] Jay Kreps commented on KAFKA-2139: -- Do you want to sketch out the design you have in mind

[jira] [Commented] (KAFKA-2141) Integrate checkstyle for Java code

2015-04-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506087#comment-14506087 ] Jay Kreps commented on KAFKA-2141: -- Do you mean adding it to cover java code that isn't i

Re: Review Request 33421: Patch for KAFKA-2114

2015-04-21 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33421/#review81102 --- Ship it! Ship It! - Sriharsha Chintalapani On April 22, 2015, 12

[jira] [Resolved] (KAFKA-2141) Integrate checkstyle for Java code

2015-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2141. -- Resolution: Not A Problem > Integrate checkstyle for Java code > ---

[jira] [Commented] (KAFKA-2141) Integrate checkstyle for Java code

2015-04-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2141?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506174#comment-14506174 ] Guozhang Wang commented on KAFKA-2141: -- I meant to create a SAMZA ticket but get mess

Re: Review Request 33410: Patch for KAFKA-2034

2015-04-21 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33410/#review81109 --- Ship it! Ship It! - Ewen Cheslack-Postava On April 21, 2015, 9 p

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-21 Thread Sriharsha Chintalapani
Hi Jay,       Thanks for the review.     1. Isn't the blocking handshake going to be a performance concern? Can we  do the handshake non-blocking instead? If anything that causes connections  to drop can incur blocking network roundtrips won't that eat up all the  network threads immediately? I gu

Re: Review Request 33417: Patch for KAFKA-2138

2015-04-21 Thread Jiangjie Qin
> On April 21, 2015, 11:56 p.m., Guozhang Wang wrote: > > This piece of logic has been quite complex and awkward to me now, for > > example in ready() a node will only not be considered if ALL of its > > partitions are either not sendable or are in the backoff period, and the > > reason we wan

Re: Metrics package discussion

2015-04-21 Thread Otis Gospodnetic
I'm veery late to this thread. I'm with Gwen about metrics being the public API (but often not treated as such, sadly). I don't know the details of internal issues around how metrics are implemented but, for selfish reasons, would hate to see MBeans change - we spent weeks contributing more t

Re: Review Request 33088: add heartbeat to coordinator

2015-04-21 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33088/#review79762 --- core/src/main/scala/kafka/api/RequestKeys.scala

RE: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-04-21 Thread Honghai Chen
Hi Sriram, One sentence of code missed, will update code review board and KIP soon. For LogSegment and FileMessageSet, must use different constructor function for existing file and new file, then the code " channel.position(math.min(channel.size().toInt, end)) " will make sure the

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-04-21 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506469#comment-14506469 ] Parth Brahmbhatt commented on KAFKA-1688: - I would like to create 3 subtasks so I

[jira] [Commented] (KAFKA-1688) Add authorization interface and naive implementation

2015-04-21 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506489#comment-14506489 ] Parth Brahmbhatt commented on KAFKA-1688: - Created reviewboard https://reviews.apa

Review Request 33431: Patch for KAFKA-1688

2015-04-21 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33431/ --- Review request for kafka. Bugs: KAFKA-1688 https://issues.apache.org/jira/b

[jira] [Updated] (KAFKA-1688) Add authorization interface and naive implementation

2015-04-21 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-1688: Attachment: KAFKA-1688.patch > Add authorization interface and naive implementation > --

[jira] [Commented] (KAFKA-2029) Improving controlled shutdown for rolling updates

2015-04-21 Thread Dmitry Bugaychenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506528#comment-14506528 ] Dmitry Bugaychenko commented on KAFKA-2029: --- The problem is that the state is di

[jira] [Commented] (KAFKA-2029) Improving controlled shutdown for rolling updates

2015-04-21 Thread Dmitry Bugaychenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14506527#comment-14506527 ] Dmitry Bugaychenko commented on KAFKA-2029: --- The problem is that the state is di

[jira] [Issue Comment Deleted] (KAFKA-2029) Improving controlled shutdown for rolling updates

2015-04-21 Thread Dmitry Bugaychenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitry Bugaychenko updated KAFKA-2029: -- Comment: was deleted (was: The problem is that the state is distributed - not only contr