Re: Perf testing flush() - issues found

2015-04-28 Thread Roshan Naik
- Event size = 1kB. - broker and client running on different machines (identical config, 32 cores, 256GB ram, 6x 1500rpm disk, 10gigEhernet) - Don't readily have number for old batch sync API for the same params. But can get it soon. However .. does it matter ? -roshan On 4/28/15 6:57 PM,

Re: Perf testing flush() - issues found

2015-04-28 Thread Joel Koshy
- What is the record size? - Is this a local setup? i.e., producer/broker running local? - Any overrides apart from batch size? E.g., linger time. - Can you establish a baseline - with the old producer's sync-send? Thanks, Joel On Wed, Apr 29, 2015 at 12:58:43AM +, Roshan Naik wrote: > Based

[jira] [Commented] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-04-28 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14518533#comment-14518533 ] Steven Zhen Wu commented on KAFKA-2121: --- Created reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-04-28 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steven Zhen Wu updated KAFKA-2121: -- Status: Patch Available (was: Reopened) > prevent potential resource leak in KafkaProducer and

[jira] [Updated] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-04-28 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steven Zhen Wu updated KAFKA-2121: -- Attachment: KAFKA-2121.patch > prevent potential resource leak in KafkaProducer and KafkaConsume

Review Request 33654: Patch for KAFKA-2121

2015-04-28 Thread Steven Wu
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33654/ --- Review request for kafka. Bugs: KAFKA-2121 https://issues.apache.org/jira/b

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

2015-04-28 Thread Haohui Mai
There are actually two advantages from using SSL/TLS compared to SASL to secure the communication channel: * Security: SSL/TLS offers Perfect Forward Secrecy when using EDH to establish encryption keys. The confidentiality of the past sessions still holds even if the long term private key of t

[jira] [Updated] (KAFKA-2151) make MockMetricsReporter a little more generic

2015-04-28 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steven Zhen Wu updated KAFKA-2151: -- Resolution: Duplicate Status: Resolved (was: Patch Available) close this one as duplica

Perf testing flush() - issues found

2015-04-28 Thread Roshan Naik
Based on recent suggestion by Joel, I am experimenting with using flush() to simulate batched-sync behavior. The essence of my single threaded producer code is : for (int i = 0; i < numRecords;) { // 1- Send a batch for(int batchCounter=0; batchCounter f = producer.send(rec

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

2015-04-28 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1660?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14518484#comment-14518484 ] Jiangjie Qin commented on KAFKA-1660: - Thanks for the reference, Jay. Totally agreed.

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

2015-04-28 Thread Sriharsha Chintalapani
Updated KIP-12 is here  https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=51809888 .  My new proposal talks about Channel having two layers one is TransportLayer and another Authentication Layer.    If users want to do  authentication without encryption they can use PlainTextTrans

[jira] [Commented] (KAFKA-2154) MetadataResponse is Empty on a Fresh Cluster

2015-04-28 Thread Emmett Butler (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14518343#comment-14518343 ] Emmett Butler commented on KAFKA-2154: -- I've been able to replicate this issue on Deb

[jira] [Commented] (KAFKA-2153) kafka-patch-review tool uploads a patch even if it is empty

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

[jira] [Updated] (KAFKA-2153) kafka-patch-review tool uploads a patch even if it is empty

2015-04-28 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2153: -- Attachment: KAFKA-2153.patch > kafka-patch-review tool uploads a patch even if it is empty > ---

[jira] [Updated] (KAFKA-2153) kafka-patch-review tool uploads a patch even if it is empty

2015-04-28 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2153: -- Status: Patch Available (was: Open) > kafka-patch-review tool uploads a patch even if it is emp

Review Request 33645: Patch for KAFKA-2153

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

[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-tabpanel&focusedCommentId=14518314#comment-14518314 ] Jay Kreps commented on KAFKA-1660: -- Here was the source of my paranoia about r/w locks co

[DISCUSS] KIP-21 Configuration Management

2015-04-28 Thread Aditya Auradkar
Hey everyone, Wrote up a KIP to update topic, client and broker configs dynamically via Zookeeper. https://cwiki.apache.org/confluence/display/KAFKA/KIP-21+-+Dynamic+Configuration Please read and provide feedback. Thanks, Aditya PS: I've intentionally kept this discussion separate from KIP-5 s

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

2015-04-28 Thread Andrii Biletskyi
Guys, A quick summary of our today's meeting. There were no additional issues/questions. The only item about which we are not 100% sure is "multiple instructions for one topic in one request" case. It was proposed by Jun to explain reasons behind not allowing users doing that again here in mailin

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

2015-04-28 Thread David Hay (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14518184#comment-14518184 ] David Hay commented on KAFKA-1835: -- [~smiklosovic] I've worked around the issue by creati

Re: Review Request 33614: Patch for KAFKA-2132

2015-04-28 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33614/ --- (Updated April 28, 2015, 9:38 p.m.) Review request for kafka. Bugs: KAFKA-213

ZooKeeper Session Timeout During Startup Caused By Long ParNewGC Pauses

2015-04-28 Thread James Lent
I have filed a bug report with Oracle on this issue today. I have not yet been assigned a bug number. I thought the issue might be of general interest to the Kafka community and I have a suggestion for a configuration change that works around the issue in case other people tun into it. First

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

2015-04-28 Thread Steven Wu
sorry for the previous empty msg. Jay's idea should work. basically, we override the close method in Serializer interface. public interface Serializer extends Closeable { @Override public void close(); } On Tue, Apr 28, 2015 at 1:10 PM, Steven Wu wrote: > > > On Tue, Apr 28, 2015 at 1:

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

2015-04-28 Thread Steven Wu
On Tue, Apr 28, 2015 at 1:03 PM, Ewen Cheslack-Postava wrote: > Good point Jay. More specifically we were already implementing without the > checked exception, we'd need to override close() in the Serializer and > Deserializer interfaces and omit the throws clause. That definitely makes > them so

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

2015-04-28 Thread Ewen Cheslack-Postava
Good point Jay. More specifically we were already implementing without the checked exception, we'd need to override close() in the Serializer and Deserializer interfaces and omit the throws clause. That definitely makes them source compatible. Not sure about binary compatibility, I couldn't find a

Re: Review Request 33634: Patch for KAFKA-2129

2015-04-28 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33634/#review81869 --- Ship it! Your analysis sounds right, looks like the missing synchro

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

2015-04-28 Thread Andrii Biletskyi
Guys, It seems that there are no open questions left so prior to our weekly call let me summarize what I'm going to implement as part of phase one for KIP-4. 1. Add 3 new Wire Protocol requests - Create-, Alter- and DeleteTopicRequest 2. Topic requests are batch requests, errors are returned per

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 wrote: > Folks, > >

[jira] [Commented] (KAFKA-1843) Metadata fetch/refresh in new producer should handle all node connection states gracefully

2015-04-28 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517659#comment-14517659 ] Ewen Cheslack-Postava commented on KAFKA-1843: -- A closely related and easily

[jira] [Commented] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-04-28 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517647#comment-14517647 ] Ewen Cheslack-Postava commented on KAFKA-2121: -- I'd give people some more tim

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

2015-04-28 Thread Stefan Miklosovic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517627#comment-14517627 ] Stefan Miklosovic commented on KAFKA-1835: -- Is there actually any workaround? I a

[jira] [Assigned] (KAFKA-2145) An option to add topic owners.

2015-04-28 Thread Neelesh Srinivas Salian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2145?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neelesh Srinivas Salian reassigned KAFKA-2145: -- Assignee: Neelesh Srinivas Salian > An option to add topic owners. > -

[jira] [Commented] (KAFKA-2145) An option to add topic owners.

2015-04-28 Thread Neelesh Srinivas Salian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517613#comment-14517613 ] Neelesh Srinivas Salian commented on KAFKA-2145: Had some clarification qu

[jira] [Commented] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-04-28 Thread Steven Zhen Wu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517596#comment-14517596 ] Steven Zhen Wu commented on KAFKA-2121: --- [~ewencp] I am ok to reverse the change on

[jira] [Updated] (KAFKA-2154) MetadataResponse is Empty on a Fresh Cluster

2015-04-28 Thread Keith Bourgoin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Keith Bourgoin updated KAFKA-2154: -- Description: When I start a fresh cluster using {{bin/kafka-server-start.sh}} and issue a Metad

[jira] [Created] (KAFKA-2154) MetadataResponse is Empty on a Fresh Cluster

2015-04-28 Thread Keith Bourgoin (JIRA)
Keith Bourgoin created KAFKA-2154: - Summary: MetadataResponse is Empty on a Fresh Cluster Key: KAFKA-2154 URL: https://issues.apache.org/jira/browse/KAFKA-2154 Project: Kafka Issue Type: Bug

[jira] [Updated] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-04-28 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tim Brooks updated KAFKA-2129: -- Assignee: Tim Brooks Status: Patch Available (was: Open) > Consumer could make multiple concurren

[jira] [Updated] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-04-28 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tim Brooks updated KAFKA-2129: -- Attachment: KAFKA-2129.patch > Consumer could make multiple concurrent metadata requests > -

[jira] [Commented] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-04-28 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517557#comment-14517557 ] Tim Brooks commented on KAFKA-2129: --- Created reviewboard https://reviews.apache.org/r/33

[jira] [Updated] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-04-28 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tim Brooks updated KAFKA-2129: -- Priority: Major (was: Minor) > Consumer could make multiple concurrent metadata requests >

Review Request 33634: Patch for KAFKA-2129

2015-04-28 Thread Tim Brooks
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33634/ --- Review request for kafka. Bugs: KAFKA-2129 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-04-28 Thread Tim Brooks (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517553#comment-14517553 ] Tim Brooks commented on KAFKA-2129: --- It kind of looks like this is a larger issue than m

Re: Adding multi-tenancy capabilities to Kafka

2015-04-28 Thread Gwen Shapira
I think recent discussion showed some need for topic namespaces - for example, Jun's use case for reserving topic names for specific users discussed under authorization. I think some discussion should happen on namespaces vs more full-fledged topic-hierarchy. I like the simplicity of namespaces, b

Adding multi-tenancy capabilities to Kafka

2015-04-28 Thread Adrian Preston
Hi all, I've been looking at how a Kafka cluster could be used be deployed so that it can be used by multiple tenants. Specifically: a scheme where the clients belonging to each tenant receive the impression they are operating against their own cluster. The ongoing security and quota work loo

[jira] [Commented] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

2015-04-28 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517489#comment-14517489 ] Aditya Auradkar commented on KAFKA-1886: [~nehanarkhede] I've updated the patch. T

[jira] [Updated] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

2015-04-28 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1886: - Attachment: KAFKA-1886_2015-04-28_10:27:39.patch > SimpleConsumer swallowing ClosedByInter

Re: Review Request 30196: Patch for KAFKA-1886

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

[jira] [Commented] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

2015-04-28 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14517479#comment-14517479 ] Aditya A Auradkar commented on KAFKA-1886: -- Updated reviewboard https://reviews.a

Re: Review Request 30196: Patch for KAFKA-1886

2015-04-28 Thread Aditya Auradkar
> On Feb. 7, 2015, 4:22 p.m., Neha Narkhede wrote: > > core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala, line 295 > > > > > > Why do you need the sleep here? We try to avoid blindly sleeping in > > Ka

Re: Review Request 30196: Patch for KAFKA-1886

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

Build failed in Jenkins: Kafka-trunk #481

2015-04-28 Thread Apache Jenkins Server
See Changes: [nehanarkhede] KAFKA-1621 : Standardize --messages option. Closes #58 -- [...truncated 1267 lines...] kafka.network.SocketServerTest > testMaxConnectionsPerIPOverrides PASSED kafka.log.O

Build failed in Jenkins: KafkaPreCommit #93

2015-04-28 Thread Apache Jenkins Server
See Changes: [nehanarkhede] KAFKA-1621 : Standardize --messages option. Closes #58 -- [...truncated 1275 lines...] kafka.network.SocketServerTest > testMaxConnectionsPerIPOverrides PASSED kafka.log

[jira] [Updated] (KAFKA-1621) Standardize --messages option in perf scripts

2015-04-28 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-1621: - Resolution: Fixed Assignee: Rekha Joshi Status: Resolved (was: Patch Available) Mer

RE: [VOTE] KIP-11- Authorization design for kafka security

2015-04-28 Thread Sun, Dapeng
Thank you for your reply, Gwen. >1. Complex rule systems can be difficult to reason about and therefore end up >being less secure. The rule "Deny always wins" is very easy to grasp. Yes, I'm agreed with your point: we should not make the rule complex. >2. We currently don't have any mechanism fo

[jira] [Updated] (KAFKA-1690) new java producer needs ssl support as a client

2015-04-28 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1690: -- Status: Patch Available (was: In Progress) > new java producer needs ssl suppor

[jira] [Updated] (KAFKA-1690) new java producer needs ssl support as a client

2015-04-28 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1690: -- Attachment: KAFKA-1690.patch > new java producer needs ssl support as a client >

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-04-28 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14516587#comment-14516587 ] Sriharsha Chintalapani commented on KAFKA-1690: --- Created reviewboard https:/

Review Request 33620: Patch for KAFKA-1690

2015-04-28 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/b

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

2015-04-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14516577#comment-14516577 ] Gwen Shapira commented on KAFKA-1928: - Moved SocketServer to use Selector. All tests

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

2015-04-28 Thread Ewen Cheslack-Postava
I suggested the change to Closeable mainly to avoid some verbosity and redundancy in the code since we have to call close() on multiple components and without a common interface, the error handling code is repeated. Using Closeable does make the code cleaner and more maintainable, but the issue was

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

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

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

2015-04-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1928: Status: Patch Available (was: In Progress) > Move kafka.network over to using the network classes i

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

2015-04-28 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1928: Attachment: KAFKA-1928_2015-04-28_00:09:40.patch > Move kafka.network over to using the network clas

Re: Review Request 33065: Patch for KAFKA-1928

2015-04-28 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/ --- (Updated April 28, 2015, 7:09 a.m.) Review request for kafka. Bugs: KAFKA-192