[jira] [Commented] (KAFKA-2217) Refactor Client Selectable Interface for Better Concurrency Options

2015-05-26 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14559567#comment-14559567 ] Jay Kreps commented on KAFKA-2217: -- This adds a lot of allocation per select call

Re: Review Request 33204: Patch for KAFKA-1646 add test cases

2015-05-20 Thread Jay Kreps
a second reviewer since this is fairly critical code. - Jay Kreps On May 11, 2015, 10 a.m., Honghai Chen wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33204

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

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552777#comment-14552777 ] Jay Kreps commented on KAFKA-1646: -- This looks good to me but it would be good to get

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

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1646: - Reviewer: Jun Rao Improve consumer read performance for Windows

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

2015-05-20 Thread Jay Kreps
changes few times... separate thread will help keep things clear for both voters and anyone who will try to figure out what happened in the future. On Wed, May 20, 2015 at 9:02 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey all, How do people feel about these [DISCUSS] threads that basically

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

2015-05-20 Thread Jay Kreps
Some information on how this will be tested would be useful. On 4/23/15 9:33 AM, Jay Kreps jay.kr...@gmail.com wrote: Yeah if we understand the optimal policy for a config we always want to set it automatically. In this case I don't think we do yet, but down the road that could

[jira] [Updated] (KAFKA-269) ./system_test/producer_perf/bin/run-test.sh without --async flag does not run

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-269: Reviewer: (was: Jay Kreps) ./system_test/producer_perf/bin/run-test.sh without --async flag does not run

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-19 Thread Jay Kreps
understandable from what I can see. What do you think? Jiangjie (Becket) Qin On 5/19/15, 11:48 AM, Jay Kreps jay.kr...@gmail.com wrote: So the alternative to consider would be to instead have max.block.ms (or something) request.timeout replication.timeout I think this better captures

[jira] [Updated] (KAFKA-2191) Measured rate should not be infinite

2015-05-19 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-2191: - Attachment: KAFKA-2191.patch Measured rate should not be infinite

Review Request 34418: Patch for KAFKA-2191

2015-05-19 Thread Jay Kreps
544e120594de78c43581a980b1e4087b4fb98ccb Diff: https://reviews.apache.org/r/34418/diff/ Testing --- Thanks, Jay Kreps

[jira] [Commented] (KAFKA-2191) Measured rate should not be infinite

2015-05-19 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550791#comment-14550791 ] Jay Kreps commented on KAFKA-2191: -- Hey [~lindong] here is a patch that shows what I'm

[jira] [Commented] (KAFKA-2191) Measured rate should not be infinite

2015-05-19 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550787#comment-14550787 ] Jay Kreps commented on KAFKA-2191: -- Created reviewboard https://reviews.apache.org/r

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-19 Thread Jay Kreps
/SampledStat.java https://reviews.apache.org/r/34170/#comment135557 Not sure if I grok why we need this while loop. Take a look at the patch I posted and see if you think that is correct. I added a test for the 0 ellapsed time case too. - Jay Kreps On May 14, 2015, 7:34 a.m., Dong Lin wrote

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-19 Thread Jay Kreps
in the record accumulator and no blocking will occur. People using the setting in the original way would now get a bit of a surprise. This may actually be okay and there is always a tradeoff between simplicity and control. -Jay On Tue, May 19, 2015 at 11:12 AM, Jay Kreps jay.kr...@gmail.com wrote: I think

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-19 Thread Jay Kreps
. request timeout is for messages already sent to kafka and the producer is waiting for them. Thanks, Mayuresh On Tue, May 19, 2015 at 11:12 AM, Jay Kreps jay.kr...@gmail.com wrote: I think this looks good. What I think is missing is an overview of the timeouts from the user's perspective

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-19 Thread Jay Kreps
I think this looks good. What I think is missing is an overview of the timeouts from the user's perspective. My worry is that it is quite complicated to reason about the current set of timeouts. Currently we have timeout.ms metadata.fetch.timeout.ms The proposed settings I think are:

Re: [VOTE] KIP-21 Dynamic Configuration

2015-05-19 Thread Jay Kreps
Hey Aditya, Two comments: 1. Yeah we need to reconcile this with the APIs in KIP-4. I think it does make sense to allow setting config during topic creation. I agree with your summary that having alter topic and alter config may be confusing, but there are also some non-config changes such as

Re: [DISCUSSION] Two behaviors in new producer

2015-05-18 Thread Jay Kreps
Hey Becket, 1. I agree that this is a problem. I think this ended up this way because there are two ways a TimeoutException can be thrown, the server can timeout or we can timeout waiting for memory. I think the complaint at the time was that it was annoying that you needed to write the same

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-15 Thread Jay Kreps
immediately on the next record call. Jay Kreps wrote: This is a really good point. It is totally possible for a metric to track activity on a topic that has no writes for a month, the first write would then cause you to cycle through a month of samples. The logic around correctly skipping

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

2015-05-15 Thread Jay Kreps
+1 -Jay On Fri, May 15, 2015 at 9:18 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface Link to Jira:

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-15 Thread Jay Kreps
immediately on the next record call. Jay Kreps wrote: This is a really good point. It is totally possible for a metric to track activity on a topic that has no writes for a month, the first write would then cause you to cycle through a month of samples. The logic around correctly skipping

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps
On May 13, 2015, 11:50 p.m., Aditya Auradkar wrote: clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java, line 45 https://reviews.apache.org/r/34170/diff/2/?file=958605#file958605line45 I think this is a good catch. Just so I understand, this

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps
immediately on the next record call. Jay Kreps wrote: This is a really good point. It is totally possible for a metric to track activity on a topic that has no writes for a month, the first write would then cause you to cycle through a month of samples. The logic around correctly skipping

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps
On May 13, 2015, 5:14 p.m., Jay Kreps wrote: clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 62 https://reviews.apache.org/r/34170/diff/1/?file=958215#file958215line62 Is this actually right? I agree you'll get discontinuities as the measured time

Re: Review Request 33731: Second Attempt to Fix KAFKA-2160

2015-05-13 Thread Jay Kreps
over ConcurrentHashMap. Adding a second hashmap with an entry for each entry really changes what it is. I recommend we not do that. If we see a bottleneck in the purgatory case let's do something specific in that case and not use Pool. - Jay Kreps On May 6, 2015, 11:31 p.m., Guozhang Wang

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps
that the value didn't fall below some threshold. clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java https://reviews.apache.org/r/34170/#comment134633 how come we are removing the event count threshold. - Jay Kreps On May 13, 2015, 3:47 p.m., Dong Lin wrote

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-12 Thread Jay Kreps
for effectively degrading the configuration for the service because of trying to keep it paired with the configuration of dynamic resources. -Todd On Mon, May 11, 2015 at 11:33 AM, Jay Kreps jay.kr...@gmail.com javascript:; wrote: I totally agree

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-11 Thread Jay Kreps
:53 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey Aditya, This is a great! A couple of comments: 1. Leaving the file config in place is definitely the least disturbance. But let's really think about getting rid of the files and just have

Re: [DISCUSSION] Reuse o.a.k.clients.NetworkClient in controller.

2015-05-07 Thread Jay Kreps
Hey guys, I haven't thought this all the way through, but I think having metadata in the NetworkClient actually does make sense. After all it is the case that all requests are directed to Kafka nodes so this is a higher level of abstraction to work at. I think the feature you are looking for is

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Jay Kreps
Hey guys, It would be nice to get a design around this. Though there are currently so many big things in flight I do wonder if we should start another parallel thing...? But working out a design can't hurt. Personally I think since one of the goals of Kafka is data integration we really want to

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-04 Thread Jay Kreps
Hey Harsha, That proposal sounds good. One minor thing--I don't think we need to have the partitioner.metadata property. Our reason for using string properties is exactly to make config extensible at runtime. So a given partitioner can add whatever properties make sense using the configure() api

Re: Review Request 31850: Patch for KAFKA-1660

2015-05-04 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31850/#review82402 --- Ship it! Ship It! - Jay Kreps On April 30, 2015, 12:37 a.m

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-04 Thread Jay Kreps
Joe, I'm not sure if I follow what you are saying. I think you are saying that unless we make all configs dynamic it will be confusing. Is that right? I somewhat agree. We are already kind of in that situation with the topic configs being dynamic, though. I think trying to make all configs

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-04 Thread Jay Kreps
it will take longer to identity which configs can be made dynamic and actually doing the work to make them so. I think that once we have reasonable agreement on the overall picture, we can implement these things piece by piece. Thanks, Aditya From: Jay

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-02 Thread Jay Kreps
Hey Aditya, This is a great! A couple of comments: 1. Leaving the file config in place is definitely the least disturbance. But let's really think about getting rid of the files and just have one config mechanism. There is always a tendency to make everything pluggable which so often just leads

Re: [DISCUSS] Using GitHub Pull Requests for contributions and code review

2015-05-02 Thread Jay Kreps
+1! -Jay On Thu, Apr 30, 2015 at 6:12 AM, Ismael Juma ism...@juma.me.uk wrote: Hi all, Kafka currently uses a combination of Review Board and JIRA for contributions and code review. In my opinion, this makes contribution and code review a bit harder than it has to be. I think the

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-01 Thread Jay Kreps
/network/SocketServer.scala https://reviews.apache.org/r/33065/#comment133004 Beautiful! So much deleted code! - Jay Kreps On May 1, 2015, 12:48 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply

Re: [DISCUSSION] java.io.Closeable in KAFKA-2121

2015-04-30 Thread Jay Kreps
them source compatible. Not sure about binary compatibility, I couldn't find a quick answer but I think it's probably still compatible. -Ewen On Tue, Apr 28, 2015 at 12:30 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey guys, You can

Re: Perf testing flush() - issues found

2015-04-29 Thread Jay Kreps
Roshan, The client allocates a batch per partition and has a hard cap on memory usage (default 32MB). When it hits that cap it waits for in-flight requests to complete to use their memory. Setting the batch size to 20M is not good--that means each partition has a 20MB array allocated for it. This

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

2015-04-28 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14518314#comment-14518314 ] Jay Kreps commented on KAFKA-1660: -- Here was the source of my paranoia about r/w locks

Re: [DISCUSSION] java.io.Closeable in KAFKA-2121

2015-04-28 Thread Jay Kreps
Hey guys, You can implement Closable without the checked exception. Having close() methods throw checked exceptions isn't very useful unless there is a way for the caller to recover. In this case there really isn't, right? -Jay On Mon, Apr 27, 2015 at 5:51 PM, Guozhang Wang wangg...@gmail.com

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-27 Thread Jay Kreps
. For offset commit requests, we would simply need a request rate metric per-client and a good default quota. Thanks, Aditya From: Jay Kreps [jay.kr...@gmail.com] Sent: Friday, April 24, 2015 3:20 PM To: dev@kafka.apache.org Subject: Re: [KIP

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

2015-04-26 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14513061#comment-14513061 ] Jay Kreps commented on KAFKA-2132: -- Hey [~singhashish], I think I may have caused

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-24 Thread Jay Kreps
developers will not have access to server metrics at all. Gwen On Tue, Apr 7, 2015 at 7:41 AM, Jay Kreps jay.kr...@gmail.com wrote: Totally. But is that the only use? What I wanted to flesh out was whether

[jira] [Resolved] (KAFKA-2148) version 0.8.2 breaks semantic versioning

2015-04-24 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-2148. -- Resolution: Not A Problem Hey Reece, this is not the same client but rather a new client

[jira] [Commented] (KAFKA-2148) version 0.8.2 breaks semantic versioning

2015-04-24 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14511304#comment-14511304 ] Jay Kreps commented on KAFKA-2148: -- Basically kafka.javaapi.producer.Producer still

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-04-23 Thread Jay Kreps
Hey Harsha, A few comments: Can you finish up the KIP there are some unfinished sentences and odd whitespace things going on. Here are the questions I think we should consider: 1. Do we need this at all given that we have the partition argument in ProducerRecord which gives full control? I

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

2015-04-23 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14509311#comment-14509311 ] Jay Kreps commented on KAFKA-1646: -- So [~waldenchen] next steps for this patch: 1. KIP

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

2015-04-23 Thread Jay Kreps
, Honghai Chen -Original Message- From: Jay Kreps [mailto:jay.kr...@gmail.com] Sent: Wednesday, April 22, 2015 5:22 AM To: dev@kafka.apache.org Subject: Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system My

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-04-23 Thread Jay Kreps
On April 23, 2015 at 9:11:33 AM, Jay Kreps (jay.kr...@gmail.com) wrote: Hey Harsha, A few comments: Can you finish up the KIP there are some unfinished sentences and odd whitespace things going on. Here are the questions I think we should consider: 1. Do we need this at all given

Re: [DISCUSS] New consumer offset commit API

2015-04-22 Thread Jay Kreps
I second Guozhang's proposal. I do think we need the callback. The current state is that for async commits you actually don't know if it succeeded. However there is a totally valid case where you do need to know if it succeeded but don't need to block, and without the callback you are stuck. I

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

2015-04-22 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14507933#comment-14507933 ] Jay Kreps commented on KAFKA-2139: -- Awesome, greatly appreciated. I agree that a separate

[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-tabpanelfocusedCommentId=14505376#comment-14505376 ] Jay Kreps commented on KAFKA-2132: -- Shouldn't the admin stuff be part of the main client

[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-tabpanelfocusedCommentId=14505329#comment-14505329 ] Jay Kreps commented on KAFKA-2132: -- Yeah it definitely makes sense to move the log4j

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

2015-04-21 Thread Jay Kreps
=DESCRIBE are there any other implied permissions like that? -Jay On Tue, Apr 21, 2015 at 1:59 PM, Jay Kreps jay.kr...@gmail.com wrote: 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

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 but

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
, 2015 at 9:23 AM, Jay Kreps jay.kr...@gmail.com wrote: 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

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

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.

[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-tabpanelfocusedCommentId=14506087#comment-14506087 ] Jay Kreps commented on KAFKA-2141: -- Do you mean adding it to cover java code that isn't

[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-tabpanelfocusedCommentId=14506078#comment-14506078 ] Jay Kreps commented on KAFKA-2029: -- I don't have a ton of context on this chunk of code

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 any

[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-tabpanelfocusedCommentId=14506085#comment-14506085 ] Jay Kreps commented on KAFKA-2139: -- Do you want to sketch out the design you have in mind

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,

[jira] [Updated] (KAFKA-2035) Add a topic config cache.

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-2035: - Status: In Progress (was: Patch Available) [~parth.brahmbhatt] Unless you object I'd prefer to either

[jira] [Updated] (KAFKA-2091) Expose a Partitioner interface in the new producer

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2091?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-2091: - Status: In Progress (was: Patch Available) Expose a Partitioner interface in the new producer

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

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1660: - Status: In Progress (was: Patch Available) Ability to call close() with a timeout on the Java Kafka

[jira] [Updated] (KAFKA-1933) Fine-grained locking in log append

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1933: - Status: In Progress (was: Patch Available) These results are very good. I'm scared of the code

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

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1646: - Status: In Progress (was: Patch Available) Improve consumer read performance for Windows

Re: Review Request 31850: Patch for KAFKA-1660

2015-04-20 Thread Jay Kreps
need to do a pretty detailed examination of the perf impact. - Jay Kreps On April 16, 2015, 6:35 p.m., Jiangjie Qin wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/31850

Re: Review Request 33065: Patch for KAFKA-1928

2015-04-20 Thread Jay Kreps
. - Jay Kreps On April 10, 2015, 4:58 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/ --- (Updated

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

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14503261#comment-14503261 ] Jay Kreps commented on KAFKA-1646: -- Cool, this looks good to me. This introduces a new

[jira] [Commented] (KAFKA-2035) Add a topic config cache.

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14503222#comment-14503222 ] Jay Kreps commented on KAFKA-2035: -- Hey [~parth.brahmbhatt] I'm not picky about

Re: Review Request 32869: Patch for KAFKA-2091

2015-04-20 Thread Jay Kreps
that configures a custom partitioner and checks that it takes effect so that this keeps working. - Jay Kreps On April 6, 2015, 12:13 a.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit

[jira] [Commented] (KAFKA-2091) Expose a Partitioner interface in the new producer

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14503174#comment-14503174 ] Jay Kreps commented on KAFKA-2091: -- This is waiting on the KIP which would be the best

[jira] [Commented] (KAFKA-2118) Cleaner cannot clean after shutdown during replaceSegments

2015-04-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2118?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14503128#comment-14503128 ] Jay Kreps commented on KAFKA-2118: -- I don't think I'll get to it, go ahead without me

Re: Review Request 33088: add heartbeat to coordinator

2015-04-18 Thread Jay Kreps
way to put all the synchronized methods in one class and pass that around that tends to stay working much longer... - Jay Kreps On April 18, 2015, 7:16 p.m., Onur Karaman wrote: --- This is an automatically generated e-mail. To reply

[jira] [Commented] (KAFKA-2035) Add a topic config cache.

2015-04-15 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14496673#comment-14496673 ] Jay Kreps commented on KAFKA-2035: -- Fair point. If we are going to model topics though

Re: Will it be possible to apply quotas based on a security principal?

2015-04-15 Thread Jay Kreps
I think this should be a fairly minor follow-up item to have the quotas key off of user rather than client id. The advantage of starting with client.id is that it decouples the security work from the quota work in the short term and provides a mechanism for those using Kafka without authentication

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

2015-04-14 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14494314#comment-14494314 ] Jay Kreps commented on KAFKA-1646: -- Oh yes, also, we'll need a patch against trunk

Re: Review Request 29091: Improve 1646 fix by add log.preallocate

2015-04-14 Thread Jay Kreps
a background process which handles the segment flush. I think trim() should just trim the excess bytes and the close() can still force the flush. core/src/main/scala/kafka/log/LogConfig.scala https://reviews.apache.org/r/29091/#comment129784 - Jay Kreps On April 14, 2015, 10:54 a.m

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-04-13 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14492598#comment-14492598 ] Jay Kreps commented on KAFKA-1910: -- Hey [~guozhang], 1. Cool. Yeah I think create

Exceptions on trunk

2015-04-12 Thread Jay Kreps
I see the following issues on trunk when doing almost anything: 1. Offset manager: it looks like this code has never been run, there is a type that tries to read the offset as a string. java.lang.ClassCastException: java.lang.Long cannot be cast to java.lang.String at

Re: Exceptions on trunk

2015-04-12 Thread Jay Kreps
Also, I just want to remind people that everything in kafka.network should really be a standalone network server, so that code block in RequestChannel that hard codes stuff about followers is really wrong. -Jay On Sun, Apr 12, 2015 at 1:12 PM, Jay Kreps jay.kr...@gmail.com wrote: I see

[jira] [Commented] (KAFKA-1910) Refactor KafkaConsumer

2015-04-12 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14491698#comment-14491698 ] Jay Kreps commented on KAFKA-1910: -- Hey [~guozhang], on the whole this made things a lot

Re: Review Request 29091: Improve 1646 fix by reduce check if Os.IsWindows

2015-04-11 Thread Jay Kreps
with an optional preallocateToSize option. 3. Move that method into FileMessageSet since that is the only place it is used and now it is a bit more idiosyncratic. - Jay Kreps On March 13, 2015, 3:12 a.m., Qianlin Xia wrote

Re: [DISCUSS] KIP-18 - JBOD Support

2015-04-11 Thread Jay Kreps
Hey Todd, The problem you pointed out is real. Unfortunately, placing by available size at creation time actually makes things worse. The original plan was to place new partitions on the disk with the most space, but consider a common case: disk 1: 500M disk 2: 0M Now say you are creating 10

Re: [DISCUSS] KIP-18 - JBOD Support

2015-04-11 Thread Jay Kreps
I think this KIP is not really about JBOD support it is just about remaining available in the presence of individual disk failures. I agree this is nice to have but in the large scope of things is this really a big deal? Kafka deployments are usually a lot smaller than Hadoop deployments so even

Re: [VOTE] KIP-13 Quotas

2015-04-08 Thread Jay Kreps
+1 Nice work. One minor nit pick: all the protocol field names are snake_case but the proposed addition is camelCase. Let's stick with the current convention. -Jay On Tue, Apr 7, 2015 at 10:47 PM, Aditya Auradkar aaurad...@linkedin.com.invalid wrote: Please vote.

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Jay Kreps
at 7:47 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey Aditya, 2. I kind of buy it, but I really like to understand the details of the use case before we make protocol changes. What changes are you proposing in the clients for monitoring and how would that be used? -Jay On Mon

Re: Few questions regarding KAFKA-1928 (reusing common network code in core)

2015-04-07 Thread Jay Kreps
Hey Gwen, Receive and Send are meant to encapsulate the idea of sending or receiving bytes. They are supposed to represent a send or receive that is in progress. The purpose of these interfaces is to allow sends and receives using transferTo when we need to do that on the server side, this

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-06 Thread Jay Kreps
concerns if I started a voting thread on the proposal after the KIP discussion tomorrow? (assuming we reach consensus on these items) Thanks, Aditya From: Jay Kreps [jay.kr...@gmail.com] Sent: Saturday, April 04, 2015 1:36 PM To: dev@kafka.apache.org

[jira] [Commented] (KAFKA-1501) transient unit tests failures due to port already in use

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14396030#comment-14396030 ] Jay Kreps commented on KAFKA-1501: -- [~gwenshap] I think the bounce tests in both

[jira] [Commented] (KAFKA-2087) TopicConfigManager javadoc references incorrect paths

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14396036#comment-14396036 ] Jay Kreps commented on KAFKA-2087: -- [~aauradkar] this doesn't look like a javadoc patch

[jira] [Commented] (KAFKA-1631) ReplicationFactor and under-replicated partitions incorrect during reassignment

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14396039#comment-14396039 ] Jay Kreps commented on KAFKA-1631: -- Is this behavior really so bad? I actually think

[jira] [Resolved] (KAFKA-1005) kafka.perf.ConsumerPerformance not shutting down consumer

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1005?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1005. -- Resolution: Fixed Committed. kafka.perf.ConsumerPerformance not shutting down consumer

[jira] [Resolved] (KAFKA-1956) New consumer client can return early from a poll() call

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1956. -- Resolution: Won't Fix New consumer client can return early from a poll() call

[jira] [Resolved] (KAFKA-1324) Debian packaging

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1324. -- Resolution: Won't Fix I think we can leave packaging out of Kafka itself since this is done in a bunch

[jira] [Resolved] (KAFKA-1049) Encoder implementations are required to provide an undocumented constructor.

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1049. -- Resolution: Won't Fix Obsolete now that we have the new clients. Encoder implementations are required

[jira] [Resolved] (KAFKA-312) Add 'reset' operation for AsyncProducerDroppedEvents

2015-04-04 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-312?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-312. - Resolution: Won't Fix Add 'reset' operation for AsyncProducerDroppedEvents

<    1   2   3   4   5   6   7   8   9   10   >