Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 3, 2015, 4:50 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 417 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line417 It's still not very clear to me how renegotiation can be supported in the middle of sends/receives. Suppose that the server initiates a handshake. This may involve the server sending some handshake bytes to the client. After this point, the server expects to read handshake bytes from the client. However, the client may still be sending some regular bytes over the socket. Sriharsha Chintalapani wrote: Client will also be middle of renegotiation right how can it send regular bytes?? Just to add if the handshakeStatus changes we set handshakeComplete to false so neither client or server can send any regular data while the renegotiation is happening. Also this is demonstrated in testRenegotiation where the client and server are sending messages renegotiation gets triggered in the middle of the sends receives and verfied after the renegotiation happens that we receive all of those messages . - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93862 --- On July 25, 2015, 7:11 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 25, 2015, 7:11 p.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Diffs - build.gradle 0abec26fb2d7be62c8a673f9ec838e926e64b2d1 checkstyle/import-control.xml 19e0659ef9385433d9f94dee43cd70a52b18c9e5 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 48fe7961e2215372d8033ece4af739ea06c6457b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 70377ae2fa46deb381139d28590ce6d4115e1adc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java bea3d737c51be77d5b5293cdd944d33b905422ba clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION
[jira] [Work started] (KAFKA-2436) log.retention.hours should be honored by LogManager
[ https://issues.apache.org/jira/browse/KAFKA-2436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2436 started by Dong Lin. --- log.retention.hours should be honored by LogManager --- Key: KAFKA-2436 URL: https://issues.apache.org/jira/browse/KAFKA-2436 Project: Kafka Issue Type: Bug Reporter: Dong Lin Assignee: Dong Lin Currently log.retention.hours is used to calculate KafkaConfig.logRetentionTimeMillis. But it is not used in LogManager to decide when to delete a log. LogManager is only using the log.retention.ms in the broker configuration. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2435) More optimally balanced partition assignment strategy
[ https://issues.apache.org/jira/browse/KAFKA-2435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698804#comment-14698804 ] Jiangjie Qin commented on KAFKA-2435: - [~noslowerdna] Thanks for the patch. I remember there was another ticket KAFKA-2019 that might solve the problem here in a different way. It basically hashes the consumer thread id instead of hash the partitions. Would that work for this case? Also, we are now in the progress of migrating from review board to github pull request. https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes Can you resubmit patch using pull request? Thanks. More optimally balanced partition assignment strategy - Key: KAFKA-2435 URL: https://issues.apache.org/jira/browse/KAFKA-2435 Project: Kafka Issue Type: Improvement Reporter: Andrew Olson Assignee: Andrew Olson Attachments: KAFKA-2435.patch While the roundrobin partition assignment strategy is an improvement over the range strategy, when the consumer topic subscriptions are not identical (previously disallowed but will be possible as of KAFKA-2172) it can produce heavily skewed assignments. As suggested [here|https://issues.apache.org/jira/browse/KAFKA-2172?focusedCommentId=14530767page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14530767] it would be nice to have a strategy that attempts to assign an equal number of partitions to each consumer in a group, regardless of how similar their individual topic subscriptions are. We can accomplish this by tracking the number of partitions assigned to each consumer, and having the partition assignment loop assign each partition to a consumer interested in that topic with the least number of partitions assigned. Additionally, we can optimize the distribution fairness by adjusting the partition assignment order: * Topics with fewer consumers are assigned first. * In the event of a tie for least consumers, the topic with more partitions is assigned first. The general idea behind these two rules is to keep the most flexible assignment choices available as long as possible by starting with the most constrained partitions/consumers. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 3, 2015, 4:50 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 306-320 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line306 It seems that the logic here can be simpler. In handshake(), we call flush at the beginning. So, it seems that when handshakeFinished(), it should always be the case that there are no remaining bytes in netWriteBuffer. So, in handshakeFinished(), it seems that we can just simply set handshakeComplete to true and turn off OP_WRITE. Also, not sure if we need to check handshakeResult.getHandshakeStatus(). Sriharsha Chintalapani wrote: Not necessarily, since we can fall to handshakeComplete from handshakeWrap. Any reason checking netWriteBuffer here is an issue? The reason to check handshakeResult is handshakeStatus is transient hence the reason we trying to check from NOT_HANDHSAKING FINISHED from main handshake method. if handshakeResult is not finished than we are not handshaking hence the reason throw exception. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93862 --- On July 25, 2015, 7:11 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 25, 2015, 7:11 p.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Diffs - build.gradle 0abec26fb2d7be62c8a673f9ec838e926e64b2d1 checkstyle/import-control.xml 19e0659ef9385433d9f94dee43cd70a52b18c9e5 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 48fe7961e2215372d8033ece4af739ea06c6457b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 70377ae2fa46deb381139d28590ce6d4115e1adc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java bea3d737c51be77d5b5293cdd944d33b905422ba clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Authenticator.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 3, 2015, 4:50 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 368-381 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line368 If handshake status is BUFFER_OVERFLOW, we will return to the caller and then to the selector. However, we may have read all incoming bytes into netReadBuffer. So, the key may never be selected again to complete the handshake. It seems that this case can never happen during handshake since we don't expect to use the appReadBuffer. Perhaps we can just assert that state is illegal when handling NEED_UNWRAP in handshake(). Sorry where you want to add assert? and which state you want to handle? - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93862 --- On July 25, 2015, 7:11 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 25, 2015, 7:11 p.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Diffs - build.gradle 0abec26fb2d7be62c8a673f9ec838e926e64b2d1 checkstyle/import-control.xml 19e0659ef9385433d9f94dee43cd70a52b18c9e5 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 48fe7961e2215372d8033ece4af739ea06c6457b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 70377ae2fa46deb381139d28590ce6d4115e1adc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java bea3d737c51be77d5b5293cdd944d33b905422ba clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Authenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java df0e6d5105ca97b7e1cb4d334ffb7b443506bd0b clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java PRE-CREATION
[jira] [Commented] (KAFKA-2437) Controller lost /controller zookeeper watcher.
[ https://issues.apache.org/jira/browse/KAFKA-2437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698910#comment-14698910 ] Jiangjie Qin commented on KAFKA-2437: - Possibly related to KAFKA-1155. I'm doing some further investigation. Controller lost /controller zookeeper watcher. -- Key: KAFKA-2437 URL: https://issues.apache.org/jira/browse/KAFKA-2437 Project: Kafka Issue Type: Bug Reporter: Jiangjie Qin Assignee: Jiangjie Qin We see this issue occasionally. The symptom is that when /controller path got deleted, the old controller does not resign so we end up having more than one controller in the cluster (although the requests from controller with old epoch will not be accepted). After checking zookeeper watcher by using wchp, it looks the zookeeper session who created the /controller path does not have a watcher on /controller. That causes the old controller not resigning. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2436) log.retention.hours should be honored by LogManager
[ https://issues.apache.org/jira/browse/KAFKA-2436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698882#comment-14698882 ] ASF GitHub Bot commented on KAFKA-2436: --- GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/142 KAFKA-2436; log.retention.hours should be honored by LogManager You can merge this pull request into a Git repository by running: $ git pull https://github.com/lindong28/kafka KAFKA-2436 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/142.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #142 commit a713d45ad4ed59440be020cc6c74efbeb2bbe54b Author: Dong Lin lindon...@gmail.com Date: 2015-08-16T23:11:58Z KAFKA-2436; log.retention.hours should be honored by LogManager log.retention.hours should be honored by LogManager --- Key: KAFKA-2436 URL: https://issues.apache.org/jira/browse/KAFKA-2436 Project: Kafka Issue Type: Bug Reporter: Dong Lin Assignee: Dong Lin Currently log.retention.hours is used to calculate KafkaConfig.logRetentionTimeMillis. But it is not used in LogManager to decide when to delete a log. LogManager is only using the log.retention.ms in the broker configuration. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] kafka pull request: KAFKA-2436; log.retention.hours should be hono...
GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/142 KAFKA-2436; log.retention.hours should be honored by LogManager You can merge this pull request into a Git repository by running: $ git pull https://github.com/lindong28/kafka KAFKA-2436 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/142.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #142 commit a713d45ad4ed59440be020cc6c74efbeb2bbe54b Author: Dong Lin lindon...@gmail.com Date: 2015-08-16T23:11:58Z KAFKA-2436; log.retention.hours should be honored by LogManager --- 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 enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Created] (KAFKA-2437) Controller lost /controller zookeeper watcher.
Jiangjie Qin created KAFKA-2437: --- Summary: Controller lost /controller zookeeper watcher. Key: KAFKA-2437 URL: https://issues.apache.org/jira/browse/KAFKA-2437 Project: Kafka Issue Type: Bug Reporter: Jiangjie Qin Assignee: Jiangjie Qin We see this issue occasionally. The symptom is that when /controller path got deleted, the old controller does not resign so we end up having more than one controller in the cluster (although the requests from controller with old epoch will not be accepted). After checking zookeeper watcher by using wchp, it looks the zookeeper session who created the /controller path does not have a watcher on /controller. That causes the old controller not resigning. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
[ https://issues.apache.org/jira/browse/KAFKA-2438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698968#comment-14698968 ] Sriharsha Chintalapani commented on KAFKA-2438: --- PR posted here https://github.com/apache/kafka/pull/143 add maxParallelForks to build.gradle to speedup tests - Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] kafka pull request: KAFKA-2438: add maxParallelForks to build.grad...
GitHub user harshach opened a pull request: https://github.com/apache/kafka/pull/143 KAFKA-2438: add maxParallelForks to build.gradle to speedup tests. You can merge this pull request into a Git repository by running: $ git pull https://github.com/harshach/kafka KAFKA-2438 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/143.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #143 commit 69b0edecc9becebc08c2a8a21777e8707fb3a564 Author: Sriharsha Chintalapani har...@hortonworks.com Date: 2015-08-17T03:29:46Z KAFKA-2438: add maxParallelForks to build.gradle to speedup tests. --- 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 enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
[ https://issues.apache.org/jira/browse/KAFKA-2438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698967#comment-14698967 ] ASF GitHub Bot commented on KAFKA-2438: --- GitHub user harshach opened a pull request: https://github.com/apache/kafka/pull/143 KAFKA-2438: add maxParallelForks to build.gradle to speedup tests. You can merge this pull request into a Git repository by running: $ git pull https://github.com/harshach/kafka KAFKA-2438 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/143.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #143 commit 69b0edecc9becebc08c2a8a21777e8707fb3a564 Author: Sriharsha Chintalapani har...@hortonworks.com Date: 2015-08-17T03:29:46Z KAFKA-2438: add maxParallelForks to build.gradle to speedup tests. add maxParallelForks to build.gradle to speedup tests - Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
[ https://issues.apache.org/jira/browse/KAFKA-2438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698982#comment-14698982 ] ASF GitHub Bot commented on KAFKA-2438: --- Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/143 add maxParallelForks to build.gradle to speedup tests - Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani Fix For: 0.8.3 With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. Before {code} Total time: 18 mins 29.806 secs {code} After {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
[ https://issues.apache.org/jira/browse/KAFKA-2438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-2438. - Resolution: Fixed Fix Version/s: 0.8.3 Issue resolved by pull request 143 [https://github.com/apache/kafka/pull/143] add maxParallelForks to build.gradle to speedup tests - Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani Fix For: 0.8.3 With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. Before {code} Total time: 18 mins 29.806 secs {code} After {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] kafka pull request: KAFKA-2438: add maxParallelForks to build.grad...
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/143 --- 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 enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSSION] Kafka 0.8.2.2 release?
BTW. I think it will be great for Apache Kafka to have a 0.8.2 release manager who's role is to cherrypick low-risk bug-fixes into the 0.8.2 branch and once enough bug fixes happened (or if sufficiently critical fixes happened) to roll out a new maintenance release (with every 3 month as a reasonable bugfix release target). This will add some predictability regarding how fast we release fixes for bugs. Gwen On Sun, Aug 16, 2015 at 8:09 PM, Jeff Holoman jholo...@cloudera.com wrote: +1 for the release and also including https://issues.apache.org/jira/browse/KAFKA-2114 Thanks Jeff On Sun, Aug 16, 2015 at 2:51 PM, Stevo Slavić ssla...@gmail.com wrote: +1 (non-binding) for 0.8.2.2 release Would be nice to include in that release new producer resiliency bug fixes https://issues.apache.org/jira/browse/KAFKA-1788 and https://issues.apache.org/jira/browse/KAFKA-2120 On Fri, Aug 14, 2015 at 4:03 PM, Gwen Shapira g...@confluent.io wrote: Will be nice to include Kafka-2308 and fix two critical snappy issues in the maintenance release. Gwen On Aug 14, 2015 6:16 AM, Grant Henke ghe...@cloudera.com wrote: Just to clarify. Will KAFKA-2189 be the only patch in the release? On Fri, Aug 14, 2015 at 7:35 AM, Manikumar Reddy ku...@nmsworks.co.in wrote: +1 for 0.8.2.2 release On Fri, Aug 14, 2015 at 5:49 PM, Ismael Juma ism...@juma.me.uk wrote: I think this is a good idea as the change is minimal on our side and it has been tested in production for some time by the reporter. Best, Ismael On Fri, Aug 14, 2015 at 1:15 PM, Jun Rao j...@confluent.io wrote: Hi, Everyone, Since the release of Kafka 0.8.2.1, a number of people have reported an issue with snappy compression ( https://issues.apache.org/jira/browse/KAFKA-2189). Basically, if they use snappy in 0.8.2.1, they will experience a 2-3X space increase. The issue has since been fixed in trunk (just a snappy jar upgrade). Since 0.8.3 is still a few months away, it may make sense to do an 0.8.2.2 release just to fix this issue. Any objections? Thanks, Jun -- Grant Henke Software Engineer | Cloudera gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke -- Jeff Holoman Systems Engineer
[jira] [Created] (KAFKA-2439) Add MirrorMakerService to ducktape system tests
Geoffrey Anderson created KAFKA-2439: Summary: Add MirrorMakerService to ducktape system tests Key: KAFKA-2439 URL: https://issues.apache.org/jira/browse/KAFKA-2439 Project: Kafka Issue Type: Sub-task Reporter: Geoffrey Anderson Assignee: Geoffrey Anderson Fix For: 0.8.3 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSSION] Kafka 0.8.2.2 release?
+1 for the release and also including https://issues.apache.org/jira/browse/KAFKA-2114 Thanks Jeff On Sun, Aug 16, 2015 at 2:51 PM, Stevo Slavić ssla...@gmail.com wrote: +1 (non-binding) for 0.8.2.2 release Would be nice to include in that release new producer resiliency bug fixes https://issues.apache.org/jira/browse/KAFKA-1788 and https://issues.apache.org/jira/browse/KAFKA-2120 On Fri, Aug 14, 2015 at 4:03 PM, Gwen Shapira g...@confluent.io wrote: Will be nice to include Kafka-2308 and fix two critical snappy issues in the maintenance release. Gwen On Aug 14, 2015 6:16 AM, Grant Henke ghe...@cloudera.com wrote: Just to clarify. Will KAFKA-2189 be the only patch in the release? On Fri, Aug 14, 2015 at 7:35 AM, Manikumar Reddy ku...@nmsworks.co.in wrote: +1 for 0.8.2.2 release On Fri, Aug 14, 2015 at 5:49 PM, Ismael Juma ism...@juma.me.uk wrote: I think this is a good idea as the change is minimal on our side and it has been tested in production for some time by the reporter. Best, Ismael On Fri, Aug 14, 2015 at 1:15 PM, Jun Rao j...@confluent.io wrote: Hi, Everyone, Since the release of Kafka 0.8.2.1, a number of people have reported an issue with snappy compression ( https://issues.apache.org/jira/browse/KAFKA-2189). Basically, if they use snappy in 0.8.2.1, they will experience a 2-3X space increase. The issue has since been fixed in trunk (just a snappy jar upgrade). Since 0.8.3 is still a few months away, it may make sense to do an 0.8.2.2 release just to fix this issue. Any objections? Thanks, Jun -- Grant Henke Software Engineer | Cloudera gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke -- Jeff Holoman Systems Engineer
[jira] [Created] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
Sriharsha Chintalapani created KAFKA-2438: - Summary: add maxParallelForks to build.gradle to speedup tests Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 17, 2015, 3:41 a.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description (updated) --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. KAFKA-1690. Add SSL support to broker, producer and consumer. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support to Kafka Broker, Producer Client. Diffs (updated) - build.gradle c7f66be00d86146b2bff6c690471690b9c4f46f8 checkstyle/import-control.xml e3f4f84c6becfd9087627f018690e1e2fc2b3bba clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 0e51d7bd461d253f4396a5b6ca7cd391658807fa clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java d35b421a515074d964c7fccb73d260b847ea5f00 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java be46b6c213ad8c6c09ad22886a5f36175ab0e13a clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java 6c317480a181678747bfb6b77e315b08668653c5 clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Authenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java df0e6d5105ca97b7e1cb4d334ffb7b443506bd0b clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java 3ca0098b8ec8cfdf81158465b2d40afc47eb6f80 clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Selectable.java 618a0fa53848ae6befea7eba39c2f3285b734494 clients/src/main/java/org/apache/kafka/common/network/Selector.java ce20111ac434eb8c74585e9c63757bb9d60a832f clients/src/main/java/org/apache/kafka/common/network/Send.java 8f6daadf6b67c3414911cda77765512131e56fd3
[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14698972#comment-14698972 ] Sriharsha Chintalapani commented on KAFKA-1690: --- Updated reviewboard https://reviews.apache.org/r/33620/diff/ against branch origin/trunk new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch, KAFKA-1690_2015-08-16_20:41:02.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1690) new java producer needs ssl support as a client
[ 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_2015-08-16_20:41:02.patch new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch, KAFKA-1690_2015-08-16_20:41:02.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-2438) add maxParallelForks to build.gradle to speedup tests
[ https://issues.apache.org/jira/browse/KAFKA-2438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-2438: -- Description: With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. Before {code} Total time: 18 mins 29.806 secs {code} After {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} was: With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} add maxParallelForks to build.gradle to speedup tests - Key: KAFKA-2438 URL: https://issues.apache.org/jira/browse/KAFKA-2438 Project: Kafka Issue Type: Bug Reporter: Sriharsha Chintalapani Assignee: Sriharsha Chintalapani With current trunk unit tests on my machine takes 16+ mins and with this patch runs about 6mins. Tested on OS X and linux. Before {code} Total time: 18 mins 29.806 secs {code} After {code} BUILD SUCCESSFUL Total time: 5 mins 37.194 secs {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSS] Client-side Assignment for New Consumer
Hey Becket, These are all fair points. Regarding running Kafka as a service, it will be good for everyone to know some numbers around topic creation and changes around # of partitions. I don't think the test usage is a good one since no one should be creating and deleting topics in a loop on a production cluster. Regarding your point about mirror maker, I think we should definitely ensure that a mirror maker with a large topic subscription list should be well supported. The reason for the half an hour stalls in LinkedIn mirror makers is due to the fact that it still uses the old consumer. The 3 major reasons for long rebalance operations in the mirror maker using the old consumer are - 1. Every rebalance operation involves many writes to ZK. For large consumer groups and large number of partitions, this adds up to several seconds 2. The view of the members in a group is inconsistent and hence there are typically several rebalance attempts. Note that this is fixed in the new design where the group membership is always consistently communicated to all members. 3. The rebalance backoff time is high (of the order of seconds). The good news is that the new design has none of these problems. As we said, we will run some tests to ensure that the mirror maker case is handled well. Thank you for the feedback! Thanks, Neha On Sat, Aug 15, 2015 at 6:17 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: Neha, Ewen and Jason, Maybe I am over concerning and I agree that it does depend on the metadata change frequency. As Neha said, a few tests will be helpful. We can see how it goes. What worries me is that in LinkedIn we are in the progress of running Kafka as a service. That means user will have the ability to create/delete topics and update their topic configurations programmatically or through UI using LinkedIn internal cloud service platform. And some automated test will also be consistently running to create topic, produce/consume some data, then clean up the topics. This brings us some new challenges for mirror maker because we need to make sure it actually copies data instead of spending too much time on rebalance. As what we see now is that each rebalance will probably take 30 seconds or more to finish even with some tricks and tuning. Another related use case I want to bring up is that today when we want to do a rolling upgrade of mirror maker (26 nodes), there will be two rounds of rebalance to bounce each node, each rebalance takes about 30 seconds. So bouncing 26 nodes takes roughly half an hour. Awkwardly, during the rolling bounce, because mirror maker is keeping rebalancing, it actually does not really copy any data! So the pipeline will literally stall for half an hour! Since we are designing the new protocol, it will also be good it we make sure this use case is addressed. Thanks, Jiangjie (Becket) Qin On Sat, Aug 15, 2015 at 10:50 AM, Neha Narkhede n...@confluent.io wrote: Becket, This is a clever approach for to ensure that only one thing communicates the metadata so even if it is stale, the entire group has the same view. However, the big assumption this makes is that the coordinator is that one process that has the ability to know the metadata for group members, which does not work for any non-consumer use case. I wonder if we may be complicating the design of 95% use cases for the remaining 5%. For instance, how many times do people create and remove topics or even add partitions? We operated LI clusters for a long time and this wasn't a frequent event that would need us to optimize this design for. Also, this is something we can easily validate by running a few tests on the patch and I suggest we wait for that. Thanks, Neha On Sat, Aug 15, 2015 at 9:14 AM, Jason Gustafson ja...@confluent.io wrote: Hey Jiangjie, I was thinking about the same problem. When metadata is changing frequently, the clients may not be able to ever find agreement on the current state. The server doesn't have this problem, as you say, because it can just take a snapshot and send that to the clients. Adding a dampening setting to the client would help if churn is sporadic, but not if it is steady. I think the metadata would have to be changing really frequently for this to be a problem practically, but this is a case where the server-side approach has an advantage. Including the metadata in the join group response would require making the subscription available to the coordinator, right? We lose a little bit of the generality of the protocol, but it might not be too bad since most use cases for reusing this protocol have a similar need for metadata (and they can always pass an empty subscription if they don't). We've gone back and forth a few times on this, and I'm generally not opposed. It might help if we try to quantify the impact of the metadata churn in practice. I
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: core/src/main/scala/kafka/server/KafkaConfig.scala, line 867 https://reviews.apache.org/r/33620/diff/13/?file=1022000#file1022000line867 Why are we using a Java Map here? Is it used in Java code? If not, then why not use Scala's Map.apply to make the code more concide and idiomatic? i.e. `Map(key1 - value1, key2 - value2...)` because we need to pass these channelConfigs to SSLFactory.java - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93110 --- On July 25, 2015, 7:11 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 25, 2015, 7:11 p.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Diffs - build.gradle 0abec26fb2d7be62c8a673f9ec838e926e64b2d1 checkstyle/import-control.xml 19e0659ef9385433d9f94dee43cd70a52b18c9e5 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 48fe7961e2215372d8033ece4af739ea06c6457b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 70377ae2fa46deb381139d28590ce6d4115e1adc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java bea3d737c51be77d5b5293cdd944d33b905422ba clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Authenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java df0e6d5105ca97b7e1cb4d334ffb7b443506bd0b clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java 3ca0098b8ec8cfdf81158465b2d40afc47eb6f80 clients/src/main/java/org/apache/kafka/common/network/PlainTextChannelBuilder.java PRE-CREATION
Re: [DISCUSSION] Kafka 0.8.2.2 release?
+1 (non-binding) for 0.8.2.2 release Would be nice to include in that release new producer resiliency bug fixes https://issues.apache.org/jira/browse/KAFKA-1788 and https://issues.apache.org/jira/browse/KAFKA-2120 On Fri, Aug 14, 2015 at 4:03 PM, Gwen Shapira g...@confluent.io wrote: Will be nice to include Kafka-2308 and fix two critical snappy issues in the maintenance release. Gwen On Aug 14, 2015 6:16 AM, Grant Henke ghe...@cloudera.com wrote: Just to clarify. Will KAFKA-2189 be the only patch in the release? On Fri, Aug 14, 2015 at 7:35 AM, Manikumar Reddy ku...@nmsworks.co.in wrote: +1 for 0.8.2.2 release On Fri, Aug 14, 2015 at 5:49 PM, Ismael Juma ism...@juma.me.uk wrote: I think this is a good idea as the change is minimal on our side and it has been tested in production for some time by the reporter. Best, Ismael On Fri, Aug 14, 2015 at 1:15 PM, Jun Rao j...@confluent.io wrote: Hi, Everyone, Since the release of Kafka 0.8.2.1, a number of people have reported an issue with snappy compression ( https://issues.apache.org/jira/browse/KAFKA-2189). Basically, if they use snappy in 0.8.2.1, they will experience a 2-3X space increase. The issue has since been fixed in trunk (just a snappy jar upgrade). Since 0.8.3 is still a few months away, it may make sense to do an 0.8.2.2 release just to fix this issue. Any objections? Thanks, Jun -- Grant Henke Software Engineer | Cloudera gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
Re: Review Request 33620: Patch for KAFKA-1690
On July 31, 2015, 4:28 p.m., Ismael Juma wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 319 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line319 This message seems a bit unclear to me. Are we saying that `getHandshakeStatus() == NOT_HANDSHAKING` even though the if/else only ensures that it's not `FINISHED`? I am calling handshakeFinished if handshakeStatus happens to be FINISHED or NOT_HANDSHAKING status and in handshakeFinished method checking to see whats the status on handshakeResult if its not FINISHED throwing an exception . There are already comments explaining why this is necessary. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93729 --- On July 25, 2015, 7:11 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 25, 2015, 7:11 p.m.) Review request for kafka. Bugs: KAFKA-1690 https://issues.apache.org/jira/browse/KAFKA-1690 Repository: kafka Description --- KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Addressing reviews. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. Fixed minor issues with the patch. KAFKA-1690. new java producer needs ssl support as a client. KAFKA-1690. new java producer needs ssl support as a client. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Broker side ssl changes. KAFKA-1684. SSL for socketServer. KAFKA-1690. Added SSLProducerSendTest and fixes to get right port for SSL. Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Post merge fixes. KAFKA-1690. Added SSLProducerSendTest. KAFKA-1690. Minor fixes based on patch review comments. Merge commit KAFKA-1690. Added SSL Consumer Test. KAFKA-1690. SSL Support. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Merge branch 'trunk' into KAFKA-1690-V1 KAFKA-1690. Addressing reviews. Removed interestOps from SSLTransportLayer. KAFKA-1690. Addressing reviews. KAFKA-1690. added staged receives to selector. KAFKA-1690. Addressing reviews. Merge branch 'trunk' into KAFKA-1690-V1 Diffs - build.gradle 0abec26fb2d7be62c8a673f9ec838e926e64b2d1 checkstyle/import-control.xml 19e0659ef9385433d9f94dee43cd70a52b18c9e5 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 2c421f42ed3fc5d61cf9c87a7eaa7bb23e26f63b clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 48fe7961e2215372d8033ece4af739ea06c6457b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 70377ae2fa46deb381139d28590ce6d4115e1adc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java bea3d737c51be77d5b5293cdd944d33b905422ba clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 03b8dd23df63a8d8a117f02eabcce4a2d48c44f7 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Authenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java df0e6d5105ca97b7e1cb4d334ffb7b443506bd0b clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java PRE-CREATION