[jira] [Commented] (KAFKA-1273) Brokers should make sure replica.fetch.max.bytes >= message.max.bytes

2014-12-02 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232479#comment-14232479 ] Sriharsha Chintalapani commented on KAFKA-1273: --- [~nehanarkhede] [~junrao] A

[jira] [Resolved] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1803. Resolution: Fixed Assignee: Dave Parfitt Thanks for the patch. +1 and committed to trunk. > UncleanLe

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Rajiv Kurian
Yeah I am kind of sad about that :(. I just mentioned it to show that there are material use cases for applications where you expose the underlying ByteBuffer (I know we were talking about byte arrays) instead of serializing/deserializing objects - performance is a big one. On Tue, Dec 2, 2014 a

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jun Rao
Rajiv, That's probably a very special use case. Note that even in the new consumer api w/o the generics, the client is only going to get the byte array back. So, you won't be able to take advantage of reusing the ByteBuffer in the underlying responses. Thanks, Jun On Tue, Dec 2, 2014 at 5:26 PM

[jira] [Commented] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Dave Parfitt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232435#comment-14232435 ] Dave Parfitt commented on KAFKA-1803: - fixed in KAFKA1803.2.patch, thanks! > UncleanL

[jira] [Updated] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Dave Parfitt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dave Parfitt updated KAFKA-1803: Attachment: KAFKA1803.2.patch > UncleanLeaderElectionEnableProp in LogConfig should be of boolean >

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Rajiv Kurian
I for one use the consumer (Simple Consumer) without any deserialization. I just take the ByteBuffer wrap it a preallocated flyweight and use it without creating any objects. I'd ideally not have to wrap this logic in a deserializer interface. For every one who does do this, it seems like a very sm

[jira] [Updated] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Dave Parfitt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dave Parfitt updated KAFKA-1803: Attachment: (was: 0001-fix-for-KAFKA-1803.patch) > UncleanLeaderElectionEnableProp in LogConfig

[jira] [Updated] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Dave Parfitt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dave Parfitt updated KAFKA-1803: Attachment: 0001-fix-for-KAFKA-1803.patch > UncleanLeaderElectionEnableProp in LogConfig should be o

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

2014-12-02 Thread Apache Jenkins Server
See

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Joel Koshy
> For (1), yes, but it's easier to make a config change than a code change. > If you are using a third party library, one may not be able to make any > code change. Doesn't that assume that all organizations have to already share the same underlying specific data type definition (e.g., UniversalAv

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232407#comment-14232407 ] Ewen Cheslack-Postava commented on KAFKA-1642: -- [~Bmis13], responses to each

[jira] [Commented] (KAFKA-1803) UncleanLeaderElectionEnableProp in LogConfig should be of boolean

2014-12-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232401#comment-14232401 ] Jun Rao commented on KAFKA-1803: Thanks for the patch. In the following, we can use the ap

[jira] [Commented] (KAFKA-1273) Brokers should make sure replica.fetch.max.bytes >= message.max.bytes

2014-12-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232393#comment-14232393 ] Jun Rao commented on KAFKA-1273: Kafka jira is configured with no reopen after close. We c

[jira] [Updated] (KAFKA-1801) Remove non-functional variable definition in log4j.properties

2014-12-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1801?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1801: --- Resolution: Fixed Fix Version/s: (was: 0.8.2) 0.8.3 Assignee: Raman Gupt

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Todd Palino
I don't think it necessarily has to be. I was thinking about that while I was typing it out, and I realized that as well. With a special broker port, my biggest concern is making sure that nothing other than a broker uses it (and so cannot bypass security controls like authentication and authorizat

Build failed in Jenkins: Kafka-trunk #345

2014-12-02 Thread Apache Jenkins Server
See Changes: [neha.narkhede] KAFKA-328 Write unit test for kafka server startup and shutdown API; reviewed by Neha Narkhede -- [...truncated 979 lines...] kafka.admin.AdminTest > testPartitionReassi

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Jun Rao
Todd, Does that imply the intra-broker protocol is always plaintext? Thanks, Jun On Tue, Dec 2, 2014 at 3:31 PM, Todd Palino wrote: > Thanks. Just to add more detail as to why I think it's a good idea to be > able to segregate traffic like that... > > One reason would just be to separate out

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Jun Rao
1. What would the format of advertised listener looks like? If we have two hosts separated by a colon, it may make parsing IP v6 harder. 3.1 Currently, the only public api that exposes requests/responses is the SimpleConsumer. Since most people probably use the high level consumer, breaking the ap

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Todd Palino
Thanks. Just to add more detail as to why I think it's a good idea to be able to segregate traffic like that... One reason would just be to separate out the intra-cluster communication to a separate port. This can allow you to run it over a different interface (for example, you could have dedicate

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Gwen Shapira
The good news is that I'm not using a map to represent protocol list. The bad news is that as mentioned in the wiki: producers, consumers and broker configuration specify "security protocol", so we'll know which host/port pair to use for communication. This implicitly assumes there will be only on

[jira] [Updated] (KAFKA-328) Write unit test for kafka server startup and shutdown API

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-328?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-328: Fix Version/s: 0.8.3 > Write unit test for kafka server startup and shutdown API > -

[jira] [Commented] (KAFKA-328) Write unit test for kafka server startup and shutdown API

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-328?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232288#comment-14232288 ] Neha Narkhede commented on KAFKA-328: - Thanks for your patch. Pushed to trunk. > Write

Re: Review Request 27818: Patch for KAFKA-328

2014-12-02 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/27818/#review63605 --- Ship it! Ship It! - Neha Narkhede On Nov. 29, 2014, 7:07 a.m., B

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Todd Palino
Gwen - just my reading of what we could expect from what you had described so far. Without having gone into implementation details, there didn't seem to be anything that would block the ability to run two ports with the same protocol configuration, at least from the way you proposed to represent it

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Gwen Shapira
Hey Todd, You say "lose the ability" - you mean this ability actually exist now? Or is this something you hope the new patch will support? Gwen On Tue, Dec 2, 2014 at 2:08 PM, Todd Palino wrote: > Leaving aside the rest of this, on #1, while I consider being able to > advertise the ports a good

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Gwen Shapira
Thanks you so much for your help here Jun! Highlighting the specific protocols is very useful. See some detailed comments below. On Tue, Dec 2, 2014 at 1:58 PM, Jun Rao wrote: > Hi, Gwen, > > Thanks for writing up the wiki. Some comments below. > > 1. To make it more general, should we support a

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jun Rao
For (1), yes, but it's easier to make a config change than a code change. If you are using a third party library, one may not be able to make any code change. For (2), it's just that if most consumers always do deserialization after getting the raw bytes, perhaps it would be better to have these t

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Roger Hoover
"It also makes it possible to do validation on the server side or make other tools that inspect or display messages (e.g. the various command line tools) and do this in an easily pluggable way across tools." I agree that it's valuable to have a standard way to plugin serialization across many tool

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Todd Palino
Leaving aside the rest of this, on #1, while I consider being able to advertise the ports a good idea, I don't want to lose the ability for maintaining multiple ports with the same protocol. For example, being able to have 2 plaintext ports, one that only brokers communicate over, and one that gene

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Joel Koshy
> The issue with a separate ser/deser library is that if it's not part of the > client API, (1) users may not use it or (2) different users may use it in > different ways. For example, you can imagine that two Avro implementations > have different ways of instantiation (since it's not enforced by t

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Rajiv Kurian
Why can't the organization package the Avro implementation with a kafka client and distribute that library though? The risk of different users supplying the kafka client with different serializer/deserializer implementations still exists. On Tue, Dec 2, 2014 at 12:11 PM, Jun Rao wrote: > Joel, R

Re: [SECURITY DISCUSSION] Refactoring Brokers to support multiple ports

2014-12-02 Thread Jun Rao
Hi, Gwen, Thanks for writing up the wiki. Some comments below. 1. To make it more general, should we support a binding and an advertised host for each protocol (e.g. plaintext, ssl, etc)? We will also need to figure out how to specify the wildcard binding host. 2. Broker format change in ZK The

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jay Kreps
Yeah totally, far from preventing it, making it easy to specify/encourage a custom serializer across your org is exactly the kind of thing I was hoping to make work well. If there is a config that gives the serializer you can just default this to what you want people to use as some kind of environm

[jira] [Commented] (KAFKA-1784) Implement a ConsumerOffsetClient library

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232165#comment-14232165 ] Neha Narkhede commented on KAFKA-1784: -- Thanks [~mgharat] and [~jjkoshy] > Implement

[jira] [Commented] (KAFKA-1784) Implement a ConsumerOffsetClient library

2014-12-02 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232086#comment-14232086 ] Mayuresh Gharat commented on KAFKA-1784: Separating out the client will not be a b

RE: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Thunder Stumpges
I'm not sure I agree with this. I feel that the need to have a consistent, well documented, shared serialization approach at the organization level is important no matter what. How you structure the API doesn't change that or make it any easier or "automatic" than before. It is still possible fo

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jun Rao
Joel, Rajiv, Thunder, The issue with a separate ser/deser library is that if it's not part of the client API, (1) users may not use it or (2) different users may use it in different ways. For example, you can imagine that two Avro implementations have different ways of instantiation (since it's no

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232073#comment-14232073 ] Neha Narkhede commented on KAFKA-1788: -- [~parth.brahmbhatt] Please go ahead. > prod

[jira] [Updated] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1788: - Reviewer: Jun Rao > producer record can stay in RecordAccumulator forever if leader is no > avail

[jira] [Commented] (KAFKA-1784) Implement a ConsumerOffsetClient library

2014-12-02 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232071#comment-14232071 ] Neha Narkhede commented on KAFKA-1784: -- [~mgharat] Are you actively working on this?

[jira] [Comment Edited] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232061#comment-14232061 ] Bhavesh Mistry edited comment on KAFKA-1642 at 12/2/14 8:04 PM:

[jira] [Commented] (KAFKA-1784) Implement a ConsumerOffsetClient library

2014-12-02 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232060#comment-14232060 ] Joel Koshy commented on KAFKA-1784: --- It does not need to be a blocker but would be good

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Bhavesh Mistry (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232061#comment-14232061 ] Bhavesh Mistry commented on KAFKA-1642: --- Hi [~ewencp], I will not have time to val

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-02 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14232054#comment-14232054 ] Parth Brahmbhatt commented on KAFKA-1788: - If noone else has a plan to work on thi

[jira] [Updated] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2014-12-02 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1788: -- Assignee: Jun Rao > producer record can stay in RecordAccumulator forever if lea

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Joel Koshy
Thanks for the follow-up Jay. I still don't quite see the issue here but maybe I just need to process this a bit more. To me "packaging up the best practice and plug it in" seems to be to expose a simple low-level API and give people the option to plug in a (possibly shared) standard serializer in

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jay Kreps
Hey Joel, you are right, we discussed this, but I think we didn't think about it as deeply as we should have. I think our take was strongly shaped by having a wrapper api at LinkedIn that DOES do the serialization transparently so I think you are thinking of the producer as just an implementation d

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231867#comment-14231867 ] Steven Zhen Wu commented on KAFKA-1642: --- [~ewencp] forgot 0.8.2 is still beta, becau

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Joel Koshy
Re: pushing complexity of dealing with objects: we're talking about just a call to a serialize method to convert the object to a byte array right? Or is there more to it? (To me) that seems less cumbersome than having to interact with parameterized types. Actually, can you explain more clearly what

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231837#comment-14231837 ] Ewen Cheslack-Postava commented on KAFKA-1642: -- [~stevenz3wu] this is already

[jira] [Commented] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-12-02 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231830#comment-14231830 ] Steven Zhen Wu commented on KAFKA-1642: --- this sounds like a serious blocker issue. o

[jira] [Commented] (KAFKA-1724) Errors after reboot in single node setup

2014-12-02 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231766#comment-14231766 ] Sriharsha Chintalapani commented on KAFKA-1724: --- [~junrao] can you please lo

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Jun Rao
Joel, Thanks for the feedback. Yes, the raw bytes interface is simpler than the Generic api. However, it just pushes the complexity of dealing with the objects to the application. We also thought about the layered approach. However, this may confuse the users since there is no single entry point

[jira] [Created] (KAFKA-1804) Kafka network thread lacks top exception handler

2014-12-02 Thread Oleg Golovin (JIRA)
Oleg Golovin created KAFKA-1804: --- Summary: Kafka network thread lacks top exception handler Key: KAFKA-1804 URL: https://issues.apache.org/jira/browse/KAFKA-1804 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1772) Add an Admin message type for request response

2014-12-02 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231534#comment-14231534 ] Andrii Biletskyi commented on KAFKA-1772: - Updated reviewboard https://reviews.apa

[jira] [Updated] (KAFKA-1772) Add an Admin message type for request response

2014-12-02 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1772?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrii Biletskyi updated KAFKA-1772: Attachment: KAFKA-1772_2014-12-02_16:23:26.patch > Add an Admin message type for request res

Re: Review Request 28175: Patch for KAFKA-1772

2014-12-02 Thread Andrii Biletskyi
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28175/ --- (Updated Dec. 2, 2014, 2:23 p.m.) Review request for kafka. Bugs: KAFKA-1772

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

2014-12-02 Thread Dmitry Pekar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231470#comment-14231470 ] Dmitry Pekar commented on KAFKA-1792: - [~nehanarkhede] It is custom algorithm tested b

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2014-12-02 Thread Qianlin Xia (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14231258#comment-14231258 ] Qianlin Xia commented on KAFKA-1646: Hi, Jun. I also came from same team with Xueqian

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-02 Thread Joel Koshy
> makes it hard to reason about what type of data is being sent to Kafka and > also makes it hard to share an implementation of the serializer. For > example, to support Avro, the serialization logic could be quite involved > since it might need to register the Avro schema in some remote registry a