Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Neha Narkhede
Hey Becket, What are the next steps on this KIP. As per your comment earlier on the thread - I do agree it makes more sense to avoid duplicate effort and plan based on new consumer. I’ll modify the KIP. Did you get a chance to think about the simplified design that we proposed earlier? Do

Re: Review Request 30482: Add the coordinator to server

2015-02-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30482/#review71562 --- Ship it! Ship It! - Jay Kreps On Feb. 6, 2015, 11:02 p.m.,

Re: [DISCUSS] ConfigDec Broker Changes on Trunk

2015-02-08 Thread Jay Kreps
I don't think we need a KIP/vote here, this is just an internal refactoring. We had said previously and noted in the document that the KIPs were just for big new features or public api changes. I am a big +1 on the idea. We'll have to be careful in the code review since it would really easy to

JIRA attack!

2015-02-08 Thread Jay Kreps
I closed about 350 redundant or obsolete issues. If I closed an issue you think is not obsolete, my apologies, just reopen. -Jay

Re: Review Request 30403: Patch for KAFKA-1906

2015-02-08 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30403/#review71557 --- config/server.properties

[jira] [Created] (KAFKA-1932) kafka topic (creation) templates

2015-02-08 Thread Ahmet AKYOL (JIRA)
Ahmet AKYOL created KAFKA-1932: -- Summary: kafka topic (creation) templates Key: KAFKA-1932 URL: https://issues.apache.org/jira/browse/KAFKA-1932 Project: Kafka Issue Type: Wish

Review Request 30763: Patch for KAFKA-1865

2015-02-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30763/ --- Review request for kafka. Bugs: KAFKA-1865

Re: Review Request 29468: Patch for KAFKA-1805

2015-02-08 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review71572 ---

[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names

2015-02-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311252#comment-14311252 ] Manikumar Reddy commented on KAFKA-1884: [~guozhang] [~jkreps] KAFKA-1919 solves

Re: JIRA attack!

2015-02-08 Thread Pradeep Gollakota
Apparently I joined this list at the right time :P On Sat, Feb 7, 2015 at 4:40 PM, Jay Kreps jay.kr...@gmail.com wrote: I closed about 350 redundant or obsolete issues. If I closed an issue you think is not obsolete, my apologies, just reopen. -Jay

Re: Review Request 29831: Patch for KAFKA-1476

2015-02-08 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29831/#review71555 --- Ship it! Thanks for attaching the latest run of the tool. I

Re: Review Request 30482: Add the coordinator to server

2015-02-08 Thread Guozhang Wang
On Feb. 1, 2015, 8:46 p.m., Guozhang Wang wrote: Not sure this stuff is actually here for review...may still be a work in progress. Overall this structure of code makes a ton of sense to me. Left some minor comments. Yes this is more of a WIP patch, but the scope of this JIRA does not

Re: Review Request 30196: Patch for KAFKA-1886

2015-02-08 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30196/#review71556 --- core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala

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

2015-02-08 Thread Jiangjie Qin
I¹m a little bit concerned about the request routers among brokers. Typically we have a dominant percentage of produce and fetch request/response. Routing them from one broker to another seems not wanted. Also I think we generally have two types of requests/responses: data related and admin

[jira] [Commented] (KAFKA-1831) Producer does not provide any information about which host the data was sent to

2015-02-08 Thread Mark Payne (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311314#comment-14311314 ] Mark Payne commented on KAFKA-1831: --- [~jkreps]: that sounds perfect! I appreciate you

[jira] [Updated] (KAFKA-1932) kafka topic (creation) templates

2015-02-08 Thread Ahmet AKYOL (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1932?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ahmet AKYOL updated KAFKA-1932: --- Description: AFAIK, the only way to create a Kafka topic (without using the default settings) is

Re: [DISCUSS] KIPs

2015-02-08 Thread Jay Kreps
A problem I am having is actually understanding which KIPs are intended to be complete proposals and which are works in progress. Joe you seem to have a bunch of these. Can you move them elsewhere until they are really fully done and ready for review and discussion? -Jay On Fri, Feb 6, 2015 at

[jira] [Commented] (KAFKA-1877) Expose version via JMX for 'new' producer

2015-02-08 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311274#comment-14311274 ] Manikumar Reddy commented on KAFKA-1877: [~jkreps] [~junrao] Currently Kafka

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

2015-02-08 Thread Jay Kreps
Hey Joe, I think this is proposing several things: 1. A new command line utility. This isn't really fully specified here. There is sample usage but I actually don't really understand what all the commands will be. Also, presumably this will replace the existing shell scripts, right? We obviously

Re: Review Request 30482: Add the coordinator to server

2015-02-08 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30482/ --- (Updated Feb. 6, 2015, 11:02 p.m.) Review request for kafka. Bugs:

[jira] [Updated] (KAFKA-1758) corrupt recovery file prevents startup

2015-02-08 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1758: - Component/s: log corrupt recovery file prevents startup --

Re: org.apache.common migration

2015-02-08 Thread Joe Stein
Argh, I just realized that the producer and consumer have already almost removed that so it wouldn't be in common but just something for the broker. Maybe later this year 0.9/1.0 item to crack into. On Sun, Feb 8, 2015 at 11:34 AM, Joe Stein joe.st...@stealth.ly wrote: Jay, Can we add

[jira] [Updated] (KAFKA-1856) Add PreCommit Patch Testing

2015-02-08 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1856: - Attachment: KAFKA-1845.result.txt really cool, just tried this out {code} python

[DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-08 Thread Jay Kreps
Following up on our previous thread on making batch send a little easier, here is a concrete proposal to add a flush() method to the producer: https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API A proposed implementation is here:

[jira] [Assigned] (KAFKA-1928) Move kafka.network over to using the network classes in org.apache.kafka.common.network

2015-02-08 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira reassigned KAFKA-1928: --- Assignee: Gwen Shapira Move kafka.network over to using the network classes in

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

2015-02-08 Thread Jay Kreps
Hey Jiangjie, Re routing support doesn't force clients to use it. Java and all existing clients would work as now where request are intelligently routed by the client, but this would lower the bar for new clients. That said I agree the case for reroute get admin commands is much stronger than

[jira] [Commented] (KAFKA-1718) Message Size Too Large error when only small messages produced with Snappy

2015-02-08 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311387#comment-14311387 ] Evan Huus commented on KAFKA-1718: -- [~guozhang], [~jkreps] my understanding is that while

[jira] [Assigned] (KAFKA-1929) Convert core kafka module to use the errors in org.apache.kafka.common.errors

2015-02-08 Thread Jeff Holoman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman reassigned KAFKA-1929: --- Assignee: Jeff Holoman Convert core kafka module to use the errors in

[jira] [Commented] (KAFKA-313) Add JSON/CSV output and looping options to ConsumerOffsetChecker

2015-02-08 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311445#comment-14311445 ] Neha Narkhede commented on KAFKA-313: - bq. If KAFKA-1476 will be committed soon, it

Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-08 Thread Gwen Shapira
Looks good to me. I like the idea of not blocking additional sends but not guaranteeing that flush() will deliver them. I assume that with linger.ms = 0, flush will just be a noop (since the queue will be empty). Is that correct? Gwen On Sun, Feb 8, 2015 at 10:25 AM, Jay Kreps

[GitHub] kafka pull request: Finer locking in log append

2015-02-08 Thread redbaron
GitHub user redbaron opened a pull request: https://github.com/apache/kafka/pull/43 Finer locking in log append This patch adds finer locking when appending to log. It breaks global append lock into 2 sequential and 1 parallel phase. Basic idea is to allow every thread

Re: Review Request 30763: Patch for KAFKA-1865

2015-02-08 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30763/#review71573 --- Minor issue with cleaning an InterruptionException, but otherwise

Re: org.apache.common migration

2015-02-08 Thread Gwen Shapira
Thanks for the background. I picked the Network classes portion of it, since I was already looking at how to refactor send/receive and friends to support extending with TLS and SASL. Having to do this in just one place will be really nice :) Gwen On Sun, Feb 8, 2015 at 7:26 AM, Jay Kreps

Re: Review Request 29893: Patch for KAFKA-1856

2015-02-08 Thread Joe Stein
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29893/#review71574 --- dev-utils/test-patch.py

[jira] [Resolved] (KAFKA-1486) Move all request/responses to use schema-utils

2015-02-08 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-1486. - Resolution: Duplicate I think this duplicated KAFKA-1927. Re-open and explain if I got it wrong

Re: org.apache.common migration

2015-02-08 Thread Joe Stein
Jay, Can we add another package (or two) to org.apache.kafka.common for metadata and consensus. We can call them something else but the idea would be to have 1 common layer for meta data information (right now we put the json into zookeeper) and 1 common layer for asynchronous watches (which we

Re: JIRA attack!

2015-02-08 Thread Gwen Shapira
This was awesome :) Peak rate of 3 per minute was reported around 3:30pm PST ;) On Sat, Feb 7, 2015 at 4:40 PM, Jay Kreps jay.kr...@gmail.com wrote: I closed about 350 redundant or obsolete issues. If I closed an issue you think is not obsolete, my apologies, just reopen. -Jay

Re: [DISCUSS] ConfigDec Broker Changes on Trunk

2015-02-08 Thread Gwen Shapira
I think the new tickets can be done in parallel, and are not an actual dependency for KAFKA-1845. Is that correct? On Sat, Feb 7, 2015 at 1:44 PM, Jay Kreps jay.kr...@gmail.com wrote: I don't think we need a KIP/vote here, this is just an internal refactoring. We had said previously and noted

Re: [DISCUSS] ConfigDec Broker Changes on Trunk

2015-02-08 Thread Jay Kreps
Yeah totally, all the cleanups should be independent, this thread just reminded me to file tickets for them. -jay On Sunday, February 8, 2015, Gwen Shapira gshap...@cloudera.com wrote: I think the new tickets can be done in parallel, and are not an actual dependency for KAFKA-1845. Is that

[jira] [Commented] (KAFKA-1845) KafkaConfig should use ConfigDef

2015-02-08 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311353#comment-14311353 ] Andrii Biletskyi commented on KAFKA-1845: - Updated reviewboard

Re: Review Request 30126: Patch for KAFKA-1845

2015-02-08 Thread Andrii Biletskyi
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30126/ --- (Updated Feb. 8, 2015, 3:05 p.m.) Review request for kafka. Bugs: KAFKA-1845

[jira] [Updated] (KAFKA-1845) KafkaConfig should use ConfigDef

2015-02-08 Thread Andrii Biletskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrii Biletskyi updated KAFKA-1845: Attachment: KAFKA-1845_2015-02-08_17:05:22.patch KafkaConfig should use ConfigDef

org.apache.common migration

2015-02-08 Thread Jay Kreps
Hey all, Someone asked about why there is code duplication between org.apache.common and core. The answer seemed like it might be useful to others, so including it here: Originally Kafka was more of a proof of concept and we didn't separate the clients from the server. LinkedIn was much smaller

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Jiangjie Qin
Hi Neha, Yes, I’ve updated the KIP so the entire KIP is based on new consumer now. I’ve put both designs with and without data channel in the KIP as I still feel we might need the data channel to provide more flexibility, especially after message handler is introduced. I’ve put my thinking of the

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

2015-02-08 Thread Maxim Ivanov (JIRA)
Maxim Ivanov created KAFKA-1933: --- Summary: Fine-grained locking in log append Key: KAFKA-1933 URL: https://issues.apache.org/jira/browse/KAFKA-1933 Project: Kafka Issue Type: Improvement

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

2015-02-08 Thread Maxim Ivanov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311527#comment-14311527 ] Maxim Ivanov commented on KAFKA-1933: - Created reviewboard

Review Request 30775: Fine-grained locking in log.append

2015-02-08 Thread Maxim Ivanov
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30775/ --- Review request for kafka. Bugs: KAFKA-1933

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

2015-02-08 Thread Maxim Ivanov (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Maxim Ivanov updated KAFKA-1933: Attachment: KAFKA-1933.patch Fine-grained locking in log append

Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer

2015-02-08 Thread Jay Kreps
Well actually in the case of linger.ms = 0 the send is still asynchronous so calling flush() blocks until all the previously sent records have completed. It doesn't speed anything up in that case, though, since they are already available to send. -Jay On Sun, Feb 8, 2015 at 10:36 AM, Gwen

[GitHub] kafka pull request: Finer locking in log append

2015-02-08 Thread redbaron
Github user redbaron closed the pull request at: https://github.com/apache/kafka/pull/43 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Created] (KAFKA-1934) Add a shutdownNow() call to new producer

2015-02-08 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-1934: --- Summary: Add a shutdownNow() call to new producer Key: KAFKA-1934 URL: https://issues.apache.org/jira/browse/KAFKA-1934 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-1447) Controlled shutdown deadlock when trying to send state updates

2015-02-08 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311599#comment-14311599 ] Jiangjie Qin commented on KAFKA-1447: - I think KAFKA-1305 solved this issue.

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Neha Narkhede
Few comments - 1. Why do we need the message handler? Do you have concrete use cases in mind? If not, we should consider adding it in the future when/if we do have use cases for it. The purpose of the mirror maker is a simple tool for setting up Kafka cluster replicas. I don't see why we need to

[jira] [Commented] (KAFKA-1908) Split brain

2015-02-08 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1908?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311617#comment-14311617 ] Jiangjie Qin commented on KAFKA-1908: - [~aozeritsky] It looks the scenario should not

Review Request 30777: Patch for KAFKA-1919

2015-02-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30777/ --- Review request for kafka. Bugs: KAFKA-1919

[jira] [Resolved] (KAFKA-1934) Add a shutdownNow() call to new producer

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1934. -- Resolution: Duplicate Add a shutdownNow() call to new producer

[jira] [Updated] (KAFKA-1934) Add a shutdownNow() call to new producer

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1934: - Issue Type: New Feature (was: Bug) Add a shutdownNow() call to new producer

Re: Review Request 29467: Patch for KAFKA-1660

2015-02-08 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/#review71595 --- I would vote for the name close(long timeout, TimeUnit unit) I

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Jay Kreps
Yeah, I second Neha's comments. The current mm code has taken something pretty simple and made it pretty scary with callbacks and wait/notify stuff. Do we believe this works? I can't tell by looking at it which is kind of bad for something important like this. I don't mean this as criticism, I

[jira] [Commented] (KAFKA-1919) Metadata request issued with no backoff in new producer if there are no topics

2015-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311630#comment-14311630 ] Guozhang Wang commented on KAFKA-1919: -- Comments for the patch: instead of calling

[jira] [Commented] (KAFKA-1934) Add a shutdownNow() call to new producer

2015-02-08 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311689#comment-14311689 ] Ewen Cheslack-Postava commented on KAFKA-1934: -- There was previous discussion

Re: Review Request 29467: Patch for KAFKA-1660

2015-02-08 Thread Jay Kreps
On Feb. 9, 2015, 1:27 a.m., Jay Kreps wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 371 https://reviews.apache.org/r/29467/diff/1/?file=802888#file802888line371 This approach will actually leak the sender thread if there are still

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

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311699#comment-14311699 ] Jay Kreps commented on KAFKA-1660: -- This is very similar to KAFKA-1659 and KAFKA-1934.

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Jiangjie Qin
Hi Jay, thanks a lot for the comments. I think this solution is better. We probably don’t need data channel anymore. It can be replaced with a list of producer if we need more sender thread. I’ll update the KIP page. The reasoning about message handler is mainly for efficiency purpose. I’m

Re: [KIP-DISCUSSION] Mirror Maker Enhancement

2015-02-08 Thread Jiangjie Qin
Thanks for the feedback, Neha. Please see inline replies. ―Jiangjie (Becket) Qin On 2/8/15, 2:40 PM, Neha Narkhede n...@confluent.io wrote: Few comments - 1. Why do we need the message handler? Do you have concrete use cases in mind? If not, we should consider adding it in the future when/if

[jira] [Commented] (KAFKA-1934) Add a shutdownNow() call to new producer

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311684#comment-14311684 ] Jay Kreps commented on KAFKA-1934: -- This is a public interface change so we should do a

[jira] [Commented] (KAFKA-1884) New Producer blocks forever for Invalid topic names

2015-02-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1884?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311632#comment-14311632 ] Guozhang Wang commented on KAFKA-1884: -- [~omkreddy] I think this is a valid point, we

[jira] [Commented] (KAFKA-1919) Metadata request issued with no backoff in new producer if there are no topics

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311683#comment-14311683 ] Jay Kreps commented on KAFKA-1919: -- That's reasonable. Patch here:

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

2015-02-08 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14311690#comment-14311690 ] Jay Kreps commented on KAFKA-1933: -- This is very interesting but also kind of scary. My