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. 18, 2015, 6:24 p.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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Diffs (updated) - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 18, 2015, 4:09 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 438 https://reviews.apache.org/r/33620/diff/17/?file=1042291#file1042291line438 Hmm, do we want to break here? It seems that we want to continue in the loop until all bytes in netReadBuffer are read. Jun, my earlier code is right and added it back. Once we expanded the appReadBuffer to currentApplicationBufferSize we want to read the contents off from appReadBuffer onto dst before we can do further unwrapping the data. If dst doesn't have any space we break. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95671 --- On Aug. 18, 2015, 6:24 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 18, 2015, 6:24 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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 18, 2015, 4:09 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 450 https://reviews.apache.org/r/33620/diff/17/?file=1042291#file1042291line450 The termination condition doesn't seem quite right. If dst has no remaining space but there are still bytes in netReadBuffer, we should return, right? Jun, we want to read everything in netReadBuffer to appReadBuffer. readFromAppBuffer will check if dst has any remaining than only trasnfers the bytes from appReadBuffer. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95671 --- On Aug. 17, 2015, 7:21 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 17, 2015, 7:21 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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 19, 2015, 1:56 a.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/utils/Utils.java, line 520 https://reviews.apache.org/r/33620/diff/18/?file=1043256#file1043256line520 second flip after `BUFFER_OVERFLOW` in `handshakeWrap` - you had mentioned offline that the javadoc/source indicated that `netWriteBuffer` is untouched if it is going to yield a `BUFFER_OVERFLOW`. I had trouble finding it - can you point me to javadoc/grepcode/equivalent? http://docs.oracle.com/javase/7/docs/api/javax/net/ssl/SSLEngine.html For example, unwrap() will return a SSLEngineResult.Status.BUFFER_OVERFLOW result if the engine determines that there is not enough destination buffer space available. Applications should call SSLSession.getApplicationBufferSize() and compare that value with the space available in the destination buffer, enlarging the buffer if necessary. Similarly, if unwrap() were to return a SSLEngineResult.Status.BUFFER_UNDERFLOW, the application should call SSLSession.getPacketBufferSize() to ensure that the source buffer has enough room to hold a record (enlarging if necessary), and then obtain more inbound data. I'll add unit test around this. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95765 --- On Aug. 19, 2015, 12:24 a.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 19, 2015, 12:24 a.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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95765 --- I haven't finished going through the latest patch, but discussed concerns so far offline. I'll continue tomorrow, but this should probably be checked-in since given the multiple reviews so far and effort to rebase. clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java (line 21) https://reviews.apache.org/r/33620/#comment150879 remove EITHER clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java (line 135) https://reviews.apache.org/r/33620/#comment150890 can drop `long x` actually.. i'll accumulate these stylistic comments and we'll handle in a follow-up. clients/src/main/java/org/apache/kafka/common/utils/Utils.java (line 520) https://reviews.apache.org/r/33620/#comment150915 second flip after `BUFFER_OVERFLOW` in `handshakeWrap` - you had mentioned offline that the javadoc/source indicated that `netWriteBuffer` is untouched if it is going to yield a `BUFFER_OVERFLOW`. I had trouble finding it - can you point me to javadoc/grepcode/equivalent? - Joel Koshy On Aug. 19, 2015, 12:24 a.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 19, 2015, 12:24 a.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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
On Aug. 19, 2015, 1:56 a.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/utils/Utils.java, line 520 https://reviews.apache.org/r/33620/diff/18/?file=1043256#file1043256line520 second flip after `BUFFER_OVERFLOW` in `handshakeWrap` - you had mentioned offline that the javadoc/source indicated that `netWriteBuffer` is untouched if it is going to yield a `BUFFER_OVERFLOW`. I had trouble finding it - can you point me to javadoc/grepcode/equivalent? Sriharsha Chintalapani wrote: http://docs.oracle.com/javase/7/docs/api/javax/net/ssl/SSLEngine.html For example, unwrap() will return a SSLEngineResult.Status.BUFFER_OVERFLOW result if the engine determines that there is not enough destination buffer space available. Applications should call SSLSession.getApplicationBufferSize() and compare that value with the space available in the destination buffer, enlarging the buffer if necessary. Similarly, if unwrap() were to return a SSLEngineResult.Status.BUFFER_UNDERFLOW, the application should call SSLSession.getPacketBufferSize() to ensure that the source buffer has enough room to hold a record (enlarging if necessary), and then obtain more inbound data. I'll add unit test around this. and in openjdk SSLEngineImpl * Check for OVERFLOW. * * To be considered: We could delay enforcing the application buffer * free space requirement until after the initial handshaking. */ if ((packetLen - Record.headerSize) ea.getAppRemaining()) { return new SSLEngineResult(Status.BUFFER_OVERFLOW, hsStatus, 0, 0); } - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95765 --- On Aug. 19, 2015, 12:24 a.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 19, 2015, 12:24 a.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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 Diffs -
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95798 --- Ship it! Looks good to me. Could you rebase? - Jun Rao On Aug. 18, 2015, 6:24 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 18, 2015, 6:24 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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
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. 19, 2015, 12:24 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. KAFKA-1690. Add SSL support for Kafka Broker, Producer and Consumer. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 Diffs (updated) - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
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(). Sriharsha Chintalapani wrote: Sorry where you want to add assert? and which state you want to handle? Basically, I am saying that in handshake(), we should never hit BUFFER_OVERFLOW since we never read anything to appReadBuffer. Had we hit BUFFER_OVERFLOW, the issue is that you may have all socket bytes read into netReadBuffer, but not all bytes in netReadBuffer have been read by SslEngine. In the next Selector.poll() call, those bytes will be handled. However, this key may never be selected since there are no more bytes from the socket. If this is the case, I'd suggest that we either assert that BUFFER_OVERFLOW should never happen when handling unwrap or at least add a comment that this should never happen. case NEED_UNWRAP: log.trace(SSLHandshake NEED_UNWRAP channelId {}, appReadBuffer pos {}, netReadBuffer pos {}, netWriteBuffer pos {}, channelId, appReadBuffer.position(), netReadBuffer.position(), netWriteBuffer.position()); handshakeResult = handshakeUnwrap(read); if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { int currentPacketBufferSize = packetBufferSize(); netReadBuffer = Utils.ensureCapacity(netReadBuffer, currentPacketBufferSize); if (netReadBuffer.position() = currentPacketBufferSize) { throw new IllegalStateException(Buffer underflow when there is available data); } } else if (handshakeResult.getStatus() == Status.BUFFER_OVERFLOW) { int currentAppBufferSize = applicationBufferSize(); appReadBuffer = Utils.ensureCapacity(appReadBuffer, currentAppBufferSize); if (appReadBuffer.position() currentAppBufferSize) { throw new IllegalStateException(Buffer underflow when available data size ( + appReadBuffer.position() + ) packet buffer size ( + currentAppBufferSize + )); } } else if (handshakeResult.getStatus() == Status.CLOSED) { throw new EOFException(SSL handshake status CLOSED during handshake UNWRAP); } - Jun --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93862 --- On Aug. 17, 2015, 7:21 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 17, 2015, 7:21 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95671 --- Ship it! Thanks for the latest patch. +1. I only have a few minor comments below. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 184) https://reviews.apache.org/r/33620/#comment150764 This sentence doesn't read well. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 438) https://reviews.apache.org/r/33620/#comment150769 Hmm, do we want to break here? It seems that we want to continue in the loop until all bytes in netReadBuffer are read. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 450) https://reviews.apache.org/r/33620/#comment150770 The termination condition doesn't seem quite right. If dst has no remaining space but there are still bytes in netReadBuffer, we should return, right? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 531) https://reviews.apache.org/r/33620/#comment150771 netReadBuffer should be netWriteBuffer, right? - Jun Rao On Aug. 17, 2015, 7:21 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 17, 2015, 7:21 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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java, line 29 https://reviews.apache.org/r/33620/diff/13/?file=1021968#file1021968line29 SSL is deprecated (https://ma.ttias.be/rfc-7568-ssl-3-0-is-now-officially-deprecated/), should we be supporting it at all? If we do support it, then we should at least include a warning. Ismael Juma wrote: I noticed that SSLv3 is actually disabled in newer JDK 8 releases by default. Sriharsha Chintalapani wrote: ? I know ssl 3.0 is deprecated. As you can see from SSLConfigs.java we enable DEFAULT_ENABLED_PROTOCOLS = TLSv1.2,TLSv1.1,TLSv1 only TLS not SSL. Yes, I understand. I was asking whether we should support those protocols at all. I'll drop this from this review though, it can be discussed separately. On July 27, 2015, 1:32 p.m., Ismael Juma wrote: clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java, line 34 https://reviews.apache.org/r/33620/diff/13/?file=1021972#file1021972line34 Why not create the principal at this point instead of in the `principal()` method? Is it a costly operation? Sriharsha Chintalapani wrote: at this point we don't haven't gone through the handshake to establish principal I think it would be nice to have a comment for things like this, but I'll drop it. On July 27, 2015, 1:32 p.m., Ismael Juma wrote: core/src/main/scala/kafka/api/FetchResponse.scala, line 82 https://reviews.apache.org/r/33620/diff/13/?file=1021998#file1021998line82 Casts are to be avoided in Scala, pattern matching is a better way to do this: `channel match { case tl: TransportLayer = pending = tl.hasPendingWrites case _ = }` However, I see that this pattern is repeated in many classes, which is not good. Assuming that we can't change `Send.writeTo` to take a `TransportLayer` (either for compatibility or because there are implementations that don't use a `TransportLayer`), we should consider introducing a utility method `hasPendingWrites(channel)` that calls `hasPendingWrites` or returns false. What do you think? Sriharsha Chintalapani wrote: The reason for doing this. Here Channel can be GatheringByteChannel or some other socketchannel. As I mentioned in one of comments , when KafkaChannel used across the system especially when we deprecate/remove BlockingChannel we can go ahead and call channel.hasPendingWrites. Ismael Juma wrote: That makes sense. Still, having a utility method would make it easier to make that change when that time comes, right? Dropping it from this review. 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...)` Sriharsha Chintalapani wrote: because we need to pass these channelConfigs to SSLFactory.java Using `asJava` from `JavaConverters` would probably be nicer, but I'll drop it from this review. - Ismael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93110 --- On Aug. 17, 2015, 3:41 a.m., Sriharsha Chintalapani wrote: --- 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 --- 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95582 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 244) https://reviews.apache.org/r/33620/#comment150610 The string `channelId` should come after `NEED_UNWRAP` for consistency with the other log messages. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 325) https://reviews.apache.org/r/33620/#comment150611 The string `channelId` should come after `FINISHED` for consistency with other log messages. Or we should remove it from all of them. clients/src/main/java/org/apache/kafka/common/network/Selector.java (line 260) https://reviews.apache.org/r/33620/#comment150612 It seems like there were some issues with the merge conflict resolution here: see `### Ancestor`, `end`, ` variabt B` in the javadoc. The reason for the conflict is that there was a commit that fixed the reference to the `poll` method, the corrected description is: ``` * lists will be cleared at the beginning of each {@link #poll(long)} call and repopulated by the call if there is * any completed I/O.``` - Ismael Juma On Aug. 17, 2015, 3:41 a.m., Sriharsha Chintalapani wrote: --- 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 --- 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 - 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
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:13 p.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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 Diffs (updated) - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review95619 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 209) https://reviews.apache.org/r/33620/#comment150714 Shouldn't this be adding WRITE rather than overriding with WRITE-only: ``` key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); ``` At the moment, during renegotiation when READ is ON, there doesn't seem to be any code that turns READ back on again if it gets turned off here. I am seeing a very intermittent (one in 100) failure in the renegotiation unit test due to this. - Rajini Sivaram On Aug. 17, 2015, 4:28 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated Aug. 17, 2015, 4:28 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 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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Diffs - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
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, 7:21 p.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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. KAFKA-1690. Add SSL Support Kafka Broker, Producer and Consumer. Diffs (updated) - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
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, 4:28 p.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. KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Merge remote-tracking branch 'refs/remotes/origin/trunk' into KAFKA-1690-V1 KAFKA-1690. Add SSL support for Kafka Brokers, Producers and Consumers. Diffs (updated) - build.gradle 983587fd0b7604c3a26fcbb6a1d63e5e470d23fe 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
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
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
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
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: 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
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(). Not necessarily, since we can fall to handshakeComplete from handshakeWrap. Any reason checking netWriteBuffer here is an issue? - 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
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. Client will also be middle of renegotiation right how can it send regular bytes?? - 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 clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.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/Selector.java, line 535 https://reviews.apache.org/r/33620/diff/13/?file=1021981#file1021981line535 Not sure why we need to check hasSend. It's possible for a channel to have both sends and receives at the same time since the NetworkClient supports pipelining. Yes on server side we don't check any of it and keep sending . - 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 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: Review Request 33620: Patch for KAFKA-1690
On Aug. 3, 2015, 4:50 p.m., Jun Rao wrote: core/src/test/scala/unit/kafka/network/SocketServerTest.scala, line 207 https://reviews.apache.org/r/33620/diff/13/?file=1022008#file1022008line207 Is this test needed given the tests we have on EchoServer? Also, do we have a test where the broker listens to multiple ports? This can be added in a followup patch though. All of SSL tests (producerSendTest, ConsumerTest) have brokers listening on both plaintext ssl. I can add test to send data over plaintext as well to these tests let me know. - 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 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
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 430-433 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line430 Is this needed? If we need to expand appReadBuffer, netReadBuffer's position won't be 0 and we can just loop back. Sorry don't follow if the dst still has space to fill we are checking if appReadBuffer has any data to add to it. - 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 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: Review Request 33620: Patch for KAFKA-1690
On July 29, 2015, 1:15 a.m., Dong Lin wrote: clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java, line 52 https://reviews.apache.org/r/33620/diff/13/?file=1021992#file1021992line52 I am not sure about this. It is probably a stupid question. Do you indeed mean to use useClientCert=false in TestSSLUtils.createSSLConfig() to test client mode? Yes. Fixed. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93332 --- 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java, line 29 https://reviews.apache.org/r/33620/diff/13/?file=1021968#file1021968line29 SSL is deprecated (https://ma.ttias.be/rfc-7568-ssl-3-0-is-now-officially-deprecated/), should we be supporting it at all? If we do support it, then we should at least include a warning. Ismael Juma wrote: I noticed that SSLv3 is actually disabled in newer JDK 8 releases by default. ? I know ssl 3.0 is deprecated. As you can see from SSLConfigs.java we enable DEFAULT_ENABLED_PROTOCOLS = TLSv1.2,TLSv1.1,TLSv1 only TLS not SSL. - 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
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 262-269 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line262 Could we transition from NEED_WRAP to NOT_HANDSHAKING directly? Or NOT_HANDSHAKING can only be transitioned from FINIHED state? NOT_HANDSHAKING can only be transitioned from FINISHED state. This is an issue SSLEngine in some JDKs. - 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 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: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93939 --- clients/src/main/java/org/apache/kafka/clients/ClientUtils.java (line 77) https://reviews.apache.org/r/33620/#comment148371 nit: move this line closer to the first line where it is used, that is, to line 79. clients/src/main/java/org/apache/kafka/clients/ClientUtils.java (lines 80 - 86) https://reviews.apache.org/r/33620/#comment148379 nit: I would switch this if-else-if to a switch case as below: ''' switch (securityProtocol) { case SecurityProtocol.SSL: channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT); break; case SecurityProtocol.PLAINTEXT: channelBuilder = new PlainTextChannelBuilder(); break; default: throw new ConfigException(Invalid SecurityProtocol + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); } ''' clients/src/main/java/org/apache/kafka/clients/NetworkClient.java (line 474) https://reviews.apache.org/r/33620/#comment148372 As you are here, you could remove this extra blank line here, no? clients/src/main/java/org/apache/kafka/clients/NetworkClient.java (line 476) https://reviews.apache.org/r/33620/#comment148374 ''isSendable'' don't look like a good method name, imo. A better one would be ''canSendRequest'' or ''canSendMoreRequest''. clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java (line 19) https://reviews.apache.org/r/33620/#comment148376 Okay, this is a helper class that holds a lot of constants. In this case, it's a best practice to add a private constructor as below: ''' private SSLConfigs() {} ''' This private constructor will make the class both non-instantiable and make it non inheritable. clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java (line 28) https://reviews.apache.org/r/33620/#comment148378 The ''public'' accessor here is unnecessary as interface methods are public by default. clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 38) https://reviews.apache.org/r/33620/#comment148377 It's a good practice to make both ''key'' and ''socketChannel'' as final fields. - Edward Ribeiro 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93862 --- Thanks for the patch. A few more comments below. build.gradle (lines 247 - 249) https://reviews.apache.org/r/33620/#comment148282 As Ismael mentioned, we got rid of scala 2.9. So this is not needed. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 151 - 152) https://reviews.apache.org/r/33620/#comment148279 We probably need to try/catch IOException as we do above? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 234 - 239) https://reviews.apache.org/r/33620/#comment148265 If handshakeStatus is NEED_UNWRAP and write is true, we will fall through to the next case. However, there may still be unflushed data. flush() won't be called when write is true. Perhaps the check for write is unnecessary since (a) flush() always handles the case when write is false; (b) since we may have done a flush in line 220 and the writable status could have changed after that, which makes the value in write stale. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 262 - 269) https://reviews.apache.org/r/33620/#comment148268 Could we transition from NEED_WRAP to NOT_HANDSHAKING directly? Or NOT_HANDSHAKING can only be transitioned from FINIHED state? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 306 - 320) https://reviews.apache.org/r/33620/#comment148281 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(). clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 357 - 360) https://reviews.apache.org/r/33620/#comment148267 Is this correct? After netReadBuffer.compact in line 375, limit is set to capacity and position is set to first unread byte. The only case when they can be equal is that we get a full capacity worth of bytes and don't read any byte during unwrap. In this case, we shouldn't empty the buffer. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 368 - 381) https://reviews.apache.org/r/33620/#comment148266 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(). clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 409) https://reviews.apache.org/r/33620/#comment148274 Agreed with Dong: Maybe change to if (netread = 0) return netread? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 417) https://reviews.apache.org/r/33620/#comment148276 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. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 430 - 433) https://reviews.apache.org/r/33620/#comment148275 Is this needed? If we need to expand appReadBuffer, netReadBuffer's position won't be 0 and we can just loop back. clients/src/main/java/org/apache/kafka/common/network/Selector.java (line 75) https://reviews.apache.org/r/33620/#comment148269 Could we add a comment on why we need to maintain this map? clients/src/main/java/org/apache/kafka/common/network/Selector.java (lines 247 - 248) https://reviews.apache.org/r/33620/#comment148272 It seems that we will need to further check whether those channels in stagedReceives are muted or not. Timeout should only be 0 if there is at least one unmuted channel in stagedReceives. clients/src/main/java/org/apache/kafka/common/network/Selector.java (lines 281 - 287) https://reviews.apache.org/r/33620/#comment148273 To avoid having to buffer unbounded number of receives in staged receives, perhaps we can choose not to read from the channel if there exist staged receives for a channel. This will help protect
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93734 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 162) https://reviews.apache.org/r/33620/#comment148140 hasRemaining doesn't work here. Hence the reason to go back to remaining() - Sriharsha Chintalapani 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 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 31, 2015, 4:32 p.m., Sriharsha Chintalapani wrote: core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala, line 218 https://reviews.apache.org/r/33620/diff/13/?file=1022004#file1022004line218 If we want to enforce this coding convention.Lets open up a new JIRA and fix the checkstyle. Its hard for contributor reviewer to catch these small issues when a big patch like this gets reviewed. Well, this is Scala code so we can't use `Checkstyle` for it. I want to introduce `ScalaStyle`, but it will involve a big patch to fix the existing consistency issues before it can fail the build. The more code we add that doesn't follow the convention, the harder it becomes. Anyway, not my call whether these issues are a blocker or not. I was pointing them out as I saw existing committers mention them during code reviews. - Ismael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93736 --- 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93735 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 417) https://reviews.apache.org/r/33620/#comment148141 Yes look at SSLSelectorTest.testRenegotiation - Sriharsha Chintalapani 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 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/SSLFactory.java
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93736 --- core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala (line 218) https://reviews.apache.org/r/33620/#comment148142 If we want to enforce this coding convention.Lets open up a new JIRA and fix the checkstyle. Its hard for contributor reviewer to catch these small issues when a big patch like this gets reviewed. - Sriharsha Chintalapani 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93729 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 149) https://reviews.apache.org/r/33620/#comment148131 Is it intentional that we don't propagate the exception here? The documentation states that an `IOException` will be thrown if an `I/O error occurs`, so it's not very clear when the error is swallowed and logged. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 162) https://reviews.apache.org/r/33620/#comment148132 `netWriteBuffer.hasRemaining` is a bit better. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 245) https://reviews.apache.org/r/33620/#comment148133 The buffer is allocated via `allocate` instead of `allocateDirect` in `ensureCapacity`. Is this what we want since we use `allocateDirect` in the constructor of this class? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 247) https://reviews.apache.org/r/33620/#comment148134 Can this really happen or we're just protecting ourselves against bugs in the SSL library implementation? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 319) https://reviews.apache.org/r/33620/#comment148135 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`? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 417) https://reviews.apache.org/r/33620/#comment148136 Do we have tests that exercise renegotiation? core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala (line 64) https://reviews.apache.org/r/33620/#comment148138 `consumers` and `producers` should be vals. core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala (line 218) https://reviews.apache.org/r/33620/#comment148139 Code convention: space after `while`. There's a few more instances of this in the file. - Ismael Juma 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 31, 2015, 4:29 p.m., Sriharsha Chintalapani wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 162 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line162 hasRemaining doesn't work here. Hence the reason to go back to remaining() OK, that's good to know. But it's unclear to me why, perhaps worth a comment? - Ismael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93734 --- 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java, line 29 https://reviews.apache.org/r/33620/diff/13/?file=1021968#file1021968line29 SSL is deprecated (https://ma.ttias.be/rfc-7568-ssl-3-0-is-now-officially-deprecated/), should we be supporting it at all? If we do support it, then we should at least include a warning. I noticed that SSLv3 is actually disabled in newer JDK 8 releases by default. - Ismael --- 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: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93639 --- clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java (lines 105 - 109) https://reviews.apache.org/r/33620/#comment148049 By generating a copy of values for use in ClientUtils.createChannelBuilder(config.values()), the new producer will explain in config.logUnused(), for example: WARN The configuration ssl.keystore.location = clientkeystore.jks was supplied but isn't a known config. (org.apache.kafka.clients.producer.ProducerConfig) Can we avoid this warning to avoid confusion to user? - Dong Lin 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 30, 2015, 10:37 p.m., Dong Lin wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 416 https://reviews.apache.org/r/33620/diff/13/?file=1021979#file1021979line416 Do you mean unwrapResult.getHandshakeStatus() == HandshakeStatus.NOT_HANDSHAKING? My misunderstanding. Never mind. - Dong --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93652 --- 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93684 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 409) https://reviews.apache.org/r/33620/#comment148099 If connection between broker and client is disconnected then socketChannel.read() == -1. Maybe change to if (netread = 0) return netread? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 481) https://reviews.apache.org/r/33620/#comment148100 Similarly, maybe break out of loop if read = 0. Otherwise we might have infinite loop here. - Dong Lin 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93332 --- clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 84) https://reviews.apache.org/r/33620/#comment147709 Typo. @throws IOException If I/O error occurs clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java (line 52) https://reviews.apache.org/r/33620/#comment147741 I am not sure about this. It is probably a stupid question. Do you indeed mean to use useClientCert=false in TestSSLUtils.createSSLConfig() to test client mode? clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java (line 181) https://reviews.apache.org/r/33620/#comment147738 Since useClientCert is not used, should it be removed? core/src/test/scala/unit/kafka/utils/TestUtils.scala (line 919) https://reviews.apache.org/r/33620/#comment147739 Should clientCert be removed since it is not used in the function? - Dong Lin 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93110 --- I did an initial pass over the code (excluding tests) and left some comments. Mostly trivial fixes, I hope. build.gradle (lines 247 - 249) https://reviews.apache.org/r/33620/#comment147361 KAFKA-2348 was merged recently and if/else for 2.9 support was removed there. It should not be restored here. clients/src/main/java/org/apache/kafka/clients/ClientUtils.java (lines 80 - 86) https://reviews.apache.org/r/33620/#comment147362 Code convention: no braces for single statement. clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java (line 106) https://reviews.apache.org/r/33620/#comment147363 Why not pass `values` to the copy constructor of `HashMap`? Then the implementation is simply `return new HashMapString, Object(values);`. clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java (line 29) https://reviews.apache.org/r/33620/#comment147364 SSL is deprecated (https://ma.ttias.be/rfc-7568-ssl-3-0-is-now-officially-deprecated/), should we be supporting it at all? If we do support it, then we should at least include a warning. clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java (line 37) https://reviews.apache.org/r/33620/#comment147365 Should we support all ciphers by default, or should we be excluding ciphers that are known to be broken? clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java (lines 64 - 66) https://reviews.apache.org/r/33620/#comment147366 Code convention: no braces used for single statement (as can be seen in the `if` a few lines above). clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java (line 34) https://reviews.apache.org/r/33620/#comment147368 Why not create the principal at this point instead of in the `principal()` method? Is it a costly operation? clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java (lines 35 - 39) https://reviews.apache.org/r/33620/#comment147370 As far as I can see, these fields can be final. Also, is it intentional for `transportLayer` to be `public`? clients/src/main/java/org/apache/kafka/common/network/PlainTextChannelBuilder.java (line 27) https://reviews.apache.org/r/33620/#comment147371 Isn't `plaintext` a single word in a security context (i.e. https://en.wikipedia.org/wiki/Plaintext)? This also affects other classes like `PlainTextTransportLayer`, etc. clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 49) https://reviews.apache.org/r/33620/#comment147382 Can be final? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 46 - 49) https://reviews.apache.org/r/33620/#comment147383 Can some of these be final? clients/src/main/java/org/apache/kafka/common/network/Selector.java (line 461) https://reviews.apache.org/r/33620/#comment147384 Should this be using `channelForId`? core/src/main/scala/kafka/api/FetchResponse.scala (line 82) https://reviews.apache.org/r/33620/#comment147373 Casts are to be avoided in Scala, pattern matching is a better way to do this: `channel match { case tl: TransportLayer = pending = tl.hasPendingWrites case _ = }` However, I see that this pattern is repeated in many classes, which is not good. Assuming that we can't change `Send.writeTo` to take a `TransportLayer` (either for compatibility or because there are implementations that don't use a `TransportLayer`), we should consider introducing a utility method `hasPendingWrites(channel)` that calls `hasPendingWrites` or returns false. What do you think? core/src/main/scala/kafka/network/SocketServer.scala (lines 52 - 63) https://reviews.apache.org/r/33620/#comment147375 Are all these vals needed at class scope? core/src/main/scala/kafka/network/SocketServer.scala (line 116) https://reviews.apache.org/r/33620/#comment147374 One typically does not specify the type of the parameter for a simple function, i.e.: `requestChannel.addResponseListener(id = processors(id).wakeup())` It's also possible to use underscore syntax to avoid declaring `id` but the name could aid readability in some cases. core/src/main/scala/kafka/network/SocketServer.scala (line 232) https://reviews.apache.org/r/33620/#comment147377 Do we really have to populate slices of the `processors` array in each `Acceptor`? Would it not be better to create the full array and then pass it to `Acceptor`? Or even perhaps pass the relevant slice. core/src/main/scala/kafka/network/SocketServer.scala (line 501) https://reviews.apache.org/r/33620/#comment147379 Code convention: space after `:`
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: core/src/main/scala/kafka/api/FetchResponse.scala, line 82 https://reviews.apache.org/r/33620/diff/13/?file=1021998#file1021998line82 Casts are to be avoided in Scala, pattern matching is a better way to do this: `channel match { case tl: TransportLayer = pending = tl.hasPendingWrites case _ = }` However, I see that this pattern is repeated in many classes, which is not good. Assuming that we can't change `Send.writeTo` to take a `TransportLayer` (either for compatibility or because there are implementations that don't use a `TransportLayer`), we should consider introducing a utility method `hasPendingWrites(channel)` that calls `hasPendingWrites` or returns false. What do you think? Sriharsha Chintalapani wrote: The reason for doing this. Here Channel can be GatheringByteChannel or some other socketchannel. As I mentioned in one of comments , when KafkaChannel used across the system especially when we deprecate/remove BlockingChannel we can go ahead and call channel.hasPendingWrites. That makes sense. Still, having a utility method would make it easier to make that change when that time comes, right? - Ismael --- 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 27, 2015, 1:32 p.m., Ismael Juma wrote: core/src/main/scala/kafka/api/FetchResponse.scala, line 82 https://reviews.apache.org/r/33620/diff/13/?file=1021998#file1021998line82 Casts are to be avoided in Scala, pattern matching is a better way to do this: `channel match { case tl: TransportLayer = pending = tl.hasPendingWrites case _ = }` However, I see that this pattern is repeated in many classes, which is not good. Assuming that we can't change `Send.writeTo` to take a `TransportLayer` (either for compatibility or because there are implementations that don't use a `TransportLayer`), we should consider introducing a utility method `hasPendingWrites(channel)` that calls `hasPendingWrites` or returns false. What do you think? The reason for doing this. Here Channel can be GatheringByteChannel or some other socketchannel. As I mentioned in one of comments , when KafkaChannel used across the system especially when we deprecate/remove BlockingChannel we can go ahead and call channel.hasPendingWrites. - 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review93177 --- clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java (line 108) https://reviews.apache.org/r/33620/#comment147469 Do you mean return copy? - Dong Lin 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 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/SSLFactory.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 411 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line411 Is renegotiation actually supported? It seems that renegotiation can happen in the middle of a regular send/receive. Sriharsha Chintalapani wrote: Yes it can happen during the middle of regular send/receive. But this need to be initiated by server which we don't have any criteria i.e we are checking any certificate expiration to trigger renegotiation. As we discussed we are going to skip this for the initial version. I enabled the test for renegotiation. On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 394-397 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line394 We talked about that before. It's possible that we read all bytes for multiple recieves into the appReadBuffer. Selector only reads off one receive at a time. If there are no more bytes from incoming network, this key may not be selected to read off the next receive in the appReadBuffer. How do we resolve that issue? Sriharsha Chintalapani wrote: yes. This is an issue which I am trying to address by adding the stagedReceives on Selector side which is to read as many availbale reads as possible in a single poll and push these into stagedReceives queue and pop one out during that poll add to completedReceives. In other polls if the selectionKey is not selected and its not mute and if there is a stagedReceive waiting we will poll that and add to completedReceives. But this logic is not working on serverside as I get selector.send IllegalStateException. I am still looking into it will update the patch once I've a solution. Please do let me know if the above description sounds right to you. Added statgedReceives on selector side to read as many reads as we can in one poll and push into deque and poll one if the channel is not selected not muted. Please take a look at the selector code. On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 361-374 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line361 Hmm, if status is ok and handshakeStatus is NEED_UNWRAP, could we get into infinite loop here? Sriharsha Chintalapani wrote: Sorry. I thought I mvoed the socketChannel read inside the loop. I missed it. Jun Rao wrote: Hmm, then we have to keep checking if the socket is readable? Changed this to read once from socketChannel. The loop exits either we read all contents from netReadBuffer or cont is false. On July 22, 2015, 3 p.m., Jun Rao wrote: checkstyle/import-control.xml, line 56 https://reviews.apache.org/r/33620/diff/12/?file=1017007#file1017007line56 Is this needed? It doesn't seem that network needs to use any protocol. Yes for tests under client EchoServer.java I am using SecurityProtocol for both PlainText and SSL. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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
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 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 (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 Diffs (updated) - 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 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/SSLFactory.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 aaf60c98c2c0f4513a8d65ee0db67953a529d598 clients/src/main/java/org/apache/kafka/common/network/Send.java 8f6daadf6b67c3414911cda77765512131e56fd3 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f
Re: Review Request 33620: Patch for KAFKA-1690
On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 231-234 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line231 Still some questions on this. 1. When handshakeStatus is not NEED_UNWRAP (e.g. FINISHED), we could have flushed all the bytes. In this case, we should turn off the write interestOps in the socket key, right? 2. When handshakeStatus is NEED_UNWRAP and write is true, we will move on to the NEED_UNWRAP case. However, in this case, there may still be unflushed bytes in netWriteBuffer. 3. When handshakeStatus transitions to FINISHED, we return to the callers. Doesn't that delay the completion of the handshake since this key may no longer be selected? Sriharsha Chintalapani wrote: 1. handshakeStatus cannot go from NEED_WRAP to FINISHED it can only go to NEED_UNWRAP. Either client or server needs to get into NEED_UNWRAP before it can reach to FINISHED status. 2. This is not true. If the handshake status need_unwrap and we are checking if flush returns true if its not than that means there are unflushed bytes netWriteBuffer. we set the interestOps to WRITE and break. 3. I am not sure I follow. If the handshake is finished the read bit is still on. So either client needs to start sending some data , at that point they set write bit on. Example metadata update request or server starts to respond to the client with responses. Handshake happens at the connection in reality client is done with handshake and starts sending some requests either its a producer or consumer. 1. During handshake, if one side does wrap, the other side has to do unwrap. So, it seems one side has to transition from NEED_WRAP to FINISHED. Also, we don't turn off write interestOps in handshakeFinished(). 2. But if write is true, the conjunction is already false and we won't call flush(). It seems that if the channel is not writtable, there is no point to call flush. 3. If handshakeStatus is FINISHED in line 231, we return to the caller. Since the channel is not ready yet, we can't do sends yet. The question is whether the key will be selected again for us to finish handshake. Since the write interestOps is still on, the key will be selected. Is that what we are relying upon? 4. I would be useful to document the state transitions in the comment. On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 361-374 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line361 Hmm, if status is ok and handshakeStatus is NEED_UNWRAP, could we get into infinite loop here? Sriharsha Chintalapani wrote: Sorry. I thought I mvoed the socketChannel read inside the loop. I missed it. Hmm, then we have to keep checking if the socket is readable? - Jun --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92599 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 46 - 57) https://reviews.apache.org/r/33620/#comment146806 Some of these fields can be `final`. - Ismael Juma On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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/SSLFactory.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 231-234 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line231 Still some questions on this. 1. When handshakeStatus is not NEED_UNWRAP (e.g. FINISHED), we could have flushed all the bytes. In this case, we should turn off the write interestOps in the socket key, right? 2. When handshakeStatus is NEED_UNWRAP and write is true, we will move on to the NEED_UNWRAP case. However, in this case, there may still be unflushed bytes in netWriteBuffer. 3. When handshakeStatus transitions to FINISHED, we return to the callers. Doesn't that delay the completion of the handshake since this key may no longer be selected? 1. handshakeStatus cannot go from NEED_WRAP to FINISHED it can only go to NEED_UNWRAP. Either client or server needs to get into NEED_UNWRAP before it can reach to FINISHED status. 2. This is not true. If the handshake status need_unwrap and we are checking if flush returns true if its not than that means there are unflushed bytes netWriteBuffer. we set the interestOps to WRITE and break. 3. I am not sure I follow. If the handshake is finished the read bit is still on. So either client needs to start sending some data , at that point they set write bit on. Example metadata update request or server starts to respond to the client with responses. Handshake happens at the connection in reality client is done with handshake and starts sending some requests either its a producer or consumer. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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
Re: Review Request 33620: Patch for KAFKA-1690
On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 361-374 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line361 Hmm, if status is ok and handshakeStatus is NEED_UNWRAP, could we get into infinite loop here? Sorry. I thought I mvoed the socketChannel read inside the loop. I missed it. On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 394-397 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line394 We talked about that before. It's possible that we read all bytes for multiple recieves into the appReadBuffer. Selector only reads off one receive at a time. If there are no more bytes from incoming network, this key may not be selected to read off the next receive in the appReadBuffer. How do we resolve that issue? yes. This is an issue which I am trying to address by adding the stagedReceives on Selector side which is to read as many availbale reads as possible in a single poll and push these into stagedReceives queue and pop one out during that poll add to completedReceives. In other polls if the selectionKey is not selected and its not mute and if there is a stagedReceive waiting we will poll that and add to completedReceives. But this logic is not working on serverside as I get selector.send IllegalStateException. I am still looking into it will update the patch once I've a solution. Please do let me know if the above description sounds right to you. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92405 --- clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java (line 61) https://reviews.apache.org/r/33620/#comment146575 Is there a JIRA ticket for removing `BlockingChannel`? If so, it may be worth mentioning the id here so that it's easy to find when the time comes for the removal to be done. clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 199) https://reviews.apache.org/r/33620/#comment146804 You can use try with resources now that we require Java 7. - Ismael Juma On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On July 22, 2015, 3 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 411 https://reviews.apache.org/r/33620/diff/12/?file=1017027#file1017027line411 Is renegotiation actually supported? It seems that renegotiation can happen in the middle of a regular send/receive. Yes it can happen during the middle of regular send/receive. But this need to be initiated by server which we don't have any criteria i.e we are checking any certificate expiration to trigger renegotiation. As we discussed we are going to skip this for the initial version. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92711 --- clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java (line 47) https://reviews.apache.org/r/33620/#comment146946 It seems that we will need to add similar logic in other Sends: MultiSend, FetchResponseSend, TopicDataSend and PartitionDataSend. - Jun Rao On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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/SSLFactory.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review92314 --- Thanks for the patch. Looks good overall. A few comments below. There are a few places where we still wrap single line statement with {}. build.gradle (lines 247 - 255) https://reviews.apache.org/r/33620/#comment146726 It seems that you reverted the changes in kafka-2323. checkstyle/import-control.xml (line 56) https://reviews.apache.org/r/33620/#comment146733 Is this needed? It doesn't seem that network needs to use any protocol. clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 47) https://reviews.apache.org/r/33620/#comment146402 Could we add @Override annotation? clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 96) https://reviews.apache.org/r/33620/#comment146404 Could we add @Override annotation? There are a few other methods in this class are like that. clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 150) https://reviews.apache.org/r/33620/#comment146408 src = srcs clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 183) https://reviews.apache.org/r/33620/#comment146409 Typo Rerturns clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 191) https://reviews.apache.org/r/33620/#comment146410 param is not SelectionKey. clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java (line 200) https://reviews.apache.org/r/33620/#comment146411 param is not SelectionKey. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 87) https://reviews.apache.org/r/33620/#comment146414 Could we add the @Override annotation to this and a few other methods in this class? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 231 - 234) https://reviews.apache.org/r/33620/#comment146446 Still some questions on this. 1. When handshakeStatus is not NEED_UNWRAP (e.g. FINISHED), we could have flushed all the bytes. In this case, we should turn off the write interestOps in the socket key, right? 2. When handshakeStatus is NEED_UNWRAP and write is true, we will move on to the NEED_UNWRAP case. However, in this case, there may still be unflushed bytes in netWriteBuffer. 3. When handshakeStatus transitions to FINISHED, we return to the callers. Doesn't that delay the completion of the handshake since this key may no longer be selected? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 245 - 246) https://reviews.apache.org/r/33620/#comment146594 Since we are not making use of appReadBuffer during handshake, could OVERFLOW ever happen? If not, perhaps we can add a comment. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 361 - 374) https://reviews.apache.org/r/33620/#comment146593 Hmm, if status is ok and handshakeStatus is NEED_UNWRAP, could we get into infinite loop here? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 394 - 397) https://reviews.apache.org/r/33620/#comment146694 We talked about that before. It's possible that we read all bytes for multiple recieves into the appReadBuffer. Selector only reads off one receive at a time. If there are no more bytes from incoming network, this key may not be selected to read off the next receive in the appReadBuffer. How do we resolve that issue? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 411) https://reviews.apache.org/r/33620/#comment146696 Is renegotiation actually supported? It seems that renegotiation can happen in the middle of a regular send/receive. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 601 - 606) https://reviews.apache.org/r/33620/#comment146697 It seems that in general, we expect addInterestOps to be called only if handshake has completed. Perhaps we can just throw an IllegalStateException if handshake hasn't completed? Ditto to removeInterestOps. clients/src/main/java/org/apache/kafka/common/network/Selector.java (lines 218 - 220) https://reviews.apache.org/r/33620/#comment146390 Coding convention: no need to wrap single line statement with {}. clients/src/main/java/org/apache/kafka/common/network/Selector.java (lines 274 - 276) https://reviews.apache.org/r/33620/#comment146391 Coding convention: no need to wrap single line statement with {}. clients/src/main/java/org/apache/kafka/common/network/Selector.java (lines 434 - 435) https://reviews.apache.org/r/33620/#comment146388 We will probably need to
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 July 20, 2015, 1:10 p.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 Diffs (updated) - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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/SSLFactory.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 aaf60c98c2c0f4513a8d65ee0db67953a529d598 clients/src/main/java/org/apache/kafka/common/network/Send.java 8f6daadf6b67c3414911cda77765512131e56fd3 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION
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 July 20, 2015, 7 p.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. Diffs (updated) - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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/SSLFactory.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 aaf60c98c2c0f4513a8d65ee0db67953a529d598 clients/src/main/java/org/apache/kafka/common/network/Send.java 8f6daadf6b67c3414911cda77765512131e56fd3 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 67 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line67 Shouldn't we only start the ssl handshake after the raw socket connection is established? SSLTransportLayer is instantiated before the connection is established. startHandshake sets the state for begin handshake but doesn't write anything to socketChannel. On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 272-275 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line272 According to the comment, shouldn't we check for NOT_HANDSHAKING too? This method gets called from main handshake method in both FINISHED NOT_HANDSHAKING cases. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89677 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 3:03 p.m., Jun Rao wrote: core/src/main/scala/kafka/network/SocketServer.scala, line 393 https://reviews.apache.org/r/33620/diff/10/?file=990801#file990801line393 This parameter doesn't seem to be used. We can remove it. I am using for metrics On June 30, 2015, 3:03 p.m., Jun Rao wrote: core/src/main/scala/kafka/network/SocketServer.scala, lines 152-153 https://reviews.apache.org/r/33620/diff/10/?file=990801#file990801line152 We may have a memory visibility issue here (which seems to exist w/o the patch). Since there is no memory barrier, the request threads may not see the latest value in the processors array. One way to fix this is to populate all processors during the instantiating SocketServer. Sorry I don't follow. We are creating acceports getting created in startup() and each of those Acceptors will create processors. why would request threads will not seee the value in processors array? - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89677 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 5:20 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/network/Channel.java, lines 148-150 https://reviews.apache.org/r/33620/diff/10/?file=990775#file990775line148 This should be caught by checkstyle - can you double-check? If we want this in our java convention we should make sure we have this included in our checkstyle xml. its not getting caught by checkstyle. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89922 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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
Re: Review Request 33620: Patch for KAFKA-1690
On June 24, 2015, 4:39 p.m., Jiangjie Qin wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 75-78 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line75 Can they be replaced by clear() method? These are set only once I don't think having another method is needed. I added comments. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89063 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 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
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 294 https://reviews.apache.org/r/33620/diff/8/?file=966806#file966806line294 Not sure why we are looping back here. If the HandshakeStatus is NEED_UNWRAP, we loop back w/o reading more data from the socket, will we get into infinite loop? Sriharsha Chintalapani wrote: answered below Moved socketChannel read inside the loop and state will change which will break the loop. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84232 --- On July 20, 2015, 7 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated July 20, 2015, 7 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. Diffs - build.gradle fb9084307ae41bbb62e32720ccd7b94f26e910c8 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 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 5:27 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java, line 31 https://reviews.apache.org/r/33620/diff/10/?file=990787#file990787line31 This may be my lack of Kafak-specific knowledge, but I'm getting a bit lost, here. I see sockets, transport layers, channels, and now transport layers... it might be good to include some javadoc on what, exactly, this interface models how that abstraction relates to other abstrations with similar-sounding (to me, at least) names. Added comments. KafkaChannel should be the default network I/O across kafka brokers or clients. Since we are in middle of transition to client side network code and we've still BlockingChannel it can be confusing till we move this part of the code as well. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89910 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION 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/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
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 179 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line179 Is this needed? It seems that the assumption is that if there is sth to write, SelectionKey.OP_WRITE should already be set and there is no need to set it again. If the engine status is need_wrap than we need to write to the socketchannel. On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 203 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line203 Hmm, if write is false, we shouldn't write to the socket through flush, right? The above condition is right but its confusing everyone. I added comments or if you I can just remove it. On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 464 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line464 Can write() be called if handshakeCompelete is false? This is to guard during the renegotiation. But since we don't have renegotiation I can remove it. On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 473-477 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line473 Can renegotiation happen in the middle of a write()? What happens to the byte that we haven't finished writing? It can and we only append to netOutBuffer the unfinished bytes will stay intact and once the renegotiation finishes the writes will resume. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89677 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java aa264202f2724907924985a5ecbe74afc4c6c04b clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java bae528d31516679bed88ee61b408f209f185a8cc
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 10:54 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 321 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line321 Actually, can you describe how this would be done (say, for dealing with revoked certificates after an client authenticates)? Per your jira comment we can use an authorizer to block the client in this case, but if you have a proposal on handling periodic renegotiation it would be useful to discuss that. I agree we don't need to implement it now. I added the SSL renegotiation as part of the latest patch. Please take a look. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION 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/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/SSLFactory.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 3:03 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 602 https://reviews.apache.org/r/33620/diff/10/?file=990783#file990783line602 Not sure if we should do the flip here. The caller may not be able to read all bytes in appReadBuffer once. When the caller calls readFromAppBuffer() again, we shouldn't do the flip again. As discussed offline. we are doing flip compact. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89677 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION 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/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/SSLFactory.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
Re: Review Request 33620: Patch for KAFKA-1690
On June 30, 2015, 5:27 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/Channel.java, lines 109-114 https://reviews.apache.org/r/33620/diff/10/?file=990775#file990775line109 So you want class `Channel` to have at most one send pending at any time-- fine. But how do I, as a caller, know whether there's a send pending? IOW, if I have a `Channel` instance I'd like to invoke send, how can I know whether it's safe to do so? I kept what we have in current selector. It does throw IllegalStateException incase if there is pending send https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/network/Selector.java#L214 On June 30, 2015, 5:27 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java, lines 67-69 https://reviews.apache.org/r/33620/diff/10/?file=990772#file990772line67 This seems a bit funny for a truststore location, since `/tmp` is usually world-writable. Does Kafka have a location in the filesystem where it stores other items writable only by the user as whom Kafka runs? I do understand the implication of /tmp but we do use that for log.dirs where actual content gets stores. Its easy for default params to use /tmp and I believe any user who is turning this ssl bit on should be able to understand there needs to be location more restrictive in permissions. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89910 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89677 --- Thanks for the patch. A few comments below. Also, we need to handle zero-copy transfer in PartitionDataSend properly. Currently, the following code won't work over SSL. val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize) clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java (lines 64 - 66) https://reviews.apache.org/r/33620/#comment142775 Would it be better to just add pending to the TransportLayer interface so that we don't have to special case for SSL? Also, would it be better to name this sth like hasPendingWrites()? clients/src/main/java/org/apache/kafka/common/network/Channel.java (line 116) https://reviews.apache.org/r/33620/#comment142349 Let's add a comment on what the expected return value is. It seems that when the full response is received, we will return a NetworkReceive. Otherwise, we will return null. We probably want to add a comment on each public method whose meaning is not obvious. clients/src/main/java/org/apache/kafka/common/network/Channel.java (lines 148 - 150) https://reviews.apache.org/r/33620/#comment142350 Our current coding convention is not to wrap single line statement with {}. Could you make a pass on this in other places? clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java (lines 48 - 49) https://reviews.apache.org/r/33620/#comment142344 Do we need to use principal builder here? Should we just return an anonymous principal? clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java (lines 90 - 92) https://reviews.apache.org/r/33620/#comment142348 Our current coding convention is not to wrap single line statement with {}. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 44) https://reviews.apache.org/r/33620/#comment142651 Could we annotate all methods in the TransportLayer interface with @Override? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 58) https://reviews.apache.org/r/33620/#comment142345 Could we document this field? In particular, how is that related to the interest ops in the selection key? When do we expect them to be the same and different? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 67) https://reviews.apache.org/r/33620/#comment142347 Shouldn't we only start the ssl handshake after the raw socket connection is established? SSLTransportLayer is instantiated before the connection is established. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 162 - 165) https://reviews.apache.org/r/33620/#comment142616 It seems that we expect if there is sth to write, the socket channel should always be writtable. Should we add an assertion on this to make sure that we are in the right state? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 169 - 172) https://reviews.apache.org/r/33620/#comment142638 Could you document the typical state transition sequence during handshake()? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 178) https://reviews.apache.org/r/33620/#comment142629 When calling flush(), sometimes we check that the socket is writable, but some other times, we don't do the check. It would be clearer if we do this consistently. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 179) https://reviews.apache.org/r/33620/#comment142615 Is this needed? It seems that the assumption is that if there is sth to write, SelectionKey.OP_WRITE should already be set and there is no need to set it again. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 203) https://reviews.apache.org/r/33620/#comment142365 Hmm, if write is false, we shouldn't write to the socket through flush, right? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 217 - 218) https://reviews.apache.org/r/33620/#comment142367 These need to use appReadBuffer. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 272 - 275) https://reviews.apache.org/r/33620/#comment142642 According to the comment, shouldn't we check for NOT_HANDSHAKING too? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 306) https://reviews.apache.org/r/33620/#comment142622 Does it matter whether to complete the writes to the socket before running the task or not? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 327)
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89922 --- I have a few other comments which I can add later or discuss offline. clients/src/main/java/org/apache/kafka/common/network/Channel.java (line 116) https://reviews.apache.org/r/33620/#comment142808 Actually in this case we can probably add a complete() method to Channel and have the caller extract the NetworkReceive only if complete clients/src/main/java/org/apache/kafka/common/network/Channel.java (lines 148 - 150) https://reviews.apache.org/r/33620/#comment142809 This should be caught by checkstyle - can you double-check? If we want this in our java convention we should make sure we have this included in our checkstyle xml. clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java (lines 48 - 49) https://reviews.apache.org/r/33620/#comment142811 Per the current code I think this is needed, but this is one of the things I was wondering about as well - perhaps the naming (authenticator) is a bit misleading? For SSL, the actual authentication takes place in the transport layer (in the SSL handshake). So plain SSL would use the default authenticator and we need to extract the principal from the transport layer. We can discuss this today. - Joel Koshy On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89910 --- clients/src/main/java/org/apache/kafka/clients/ClientUtils.java (lines 80 - 84) https://reviews.apache.org/r/33620/#comment142787 If you add a new enum value, and forget to update this if/then/else statement, it will silently return a `PlainTextChannelBuilder`, which is probably not what you want. Perhaps you should at least warn the caller, if not throw, if `securityProtocol` is neither ssl nor plaintext? clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java (lines 67 - 69) https://reviews.apache.org/r/33620/#comment142788 This seems a bit funny for a truststore location, since `/tmp` is usually world-writable. Does Kafka have a location in the filesystem where it stores other items writable only by the user as whom Kafka runs? clients/src/main/java/org/apache/kafka/common/network/Authenticator.java (lines 38 - 40) https://reviews.apache.org/r/33620/#comment142789 From what shall implementations derive the `Principal`? The method takes no parameters clients/src/main/java/org/apache/kafka/common/network/Channel.java (lines 109 - 114) https://reviews.apache.org/r/33620/#comment142796 So you want class `Channel` to have at most one send pending at any time-- fine. But how do I, as a caller, know whether there's a send pending? IOW, if I have a `Channel` instance I'd like to invoke send, how can I know whether it's safe to do so? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (lines 75 - 79) https://reviews.apache.org/r/33620/#comment142812 While correct, I wonder if this is going to confuse your users? Suppose the desired setting is want client auth. I could see a user defining `SSL_NEED_CLIENT_AUTH_CONFIG` to false, and `SSL_WNAT_CLIENT_AUTH_CONFIG` to true (which would of course, result is no client auth being requested). Why not correct the (broken, IMHO) JSSE interface and use a single configuration named, say, `SSL_CLIENT_AUTH_CONFIG` which may be one of REQUIRED, REQUESTED, or NONE? In your implementation, you can deal with the non-intuitive JSSE interface. clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java (line 31) https://reviews.apache.org/r/33620/#comment142821 This may be my lack of Kafak-specific knowledge, but I'm getting a bit lost, here. I see sockets, transport layers, channels, and now transport layers... it might be good to include some javadoc on what, exactly, this interface models how that abstraction relates to other abstrations with similar-sounding (to me, at least) names. - Michael Herstine On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review89063 --- Thanks for the patch, Sriharsha. A few comments below. clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java (lines 52 - 53) https://reviews.apache.org/r/33620/#comment141700 I read some java doc but still not quite understand how are those two parameteres going to be used. Can you elaborate the usage of that a little bit? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 40) https://reviews.apache.org/r/33620/#comment141672 Nit pick, keyStore and trustStore? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 111) https://reviews.apache.org/r/33620/#comment141673 We already have a default algorithm set in config, do we still need to set default here? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (lines 135 - 138) https://reviews.apache.org/r/33620/#comment141674 Should we mention in the doc that if NeedClientAuth is set, it will be used, otherwise WantClientAuth is used. So if user knows what they get when set both to be true. clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 138) https://reviews.apache.org/r/33620/#comment141686 setWantClientAuth? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (lines 140 - 143) https://reviews.apache.org/r/33620/#comment141685 Any reason why we only set the endpoint identification algorithm on client side? Also, it looks the SSLParameters actually contains: enabledCipherSuites, enabledProtocols, need clientAuth, wantClientAuth, can we just use one SSLParameter instead of calling different methods? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 156) https://reviews.apache.org/r/33620/#comment141668 Can we put these checks in checkKeyStoreConfigs()? clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java (line 167) https://reviews.apache.org/r/33620/#comment141671 ditto above. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 75 - 78) https://reviews.apache.org/r/33620/#comment141703 Can they be replaced by clear() method? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (line 101) https://reviews.apache.org/r/33620/#comment141706 It seems we just removed the OP_CONNECT operation. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java (lines 127 - 139) https://reviews.apache.org/r/33620/#comment141794 Not sure if it matters, but JSSE reference guide suggested to have a loop checking handshake state when shutdown. - Jiangjie Qin On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 231 https://reviews.apache.org/r/33620/diff/8/?file=966794#file966794line231 Do we need config.values? Could we just pass in config.originals() as we do for the serializer? config.originals() contains the user configured properties only it doesn't have defaults defined in ProducerConfig. On the side note shouldn't we use config.values() everywhere? - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84232 --- On June 4, 2015, 1:52 a.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 4, 2015, 1:52 a.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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/ByteBufferReceive.java 129ae827bccbd982ad93d56e46c6f5c46f147fe0 clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java c8213e156ec9c9af49ee09f5238492318516aaa3 clients/src/main/java/org/apache/kafka/common/network/Channel.java PRE-CREATION 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/NetworkReceive.java fc0d168324aaebb97065b0aafbd547a1994d76a7 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/Receive.java 4e33078c1eec834bd74aabcb5fc69f18c9d6d52a clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/Send.java 5d321a09e470166a1c33639cf0cab26a3bce98ec
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 June 23, 2015, 8:18 p.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. Diffs (updated) - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION 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/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/SSLFactory.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 4aee214b24fd990be003adc36d675f015bf22fe6 clients/src/main/java/org/apache/kafka/common/network/Send.java 8f6daadf6b67c3414911cda77765512131e56fd3 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 231 https://reviews.apache.org/r/33620/diff/8/?file=966794#file966794line231 Do we need config.values? Could we just pass in config.originals() as we do for the serializer? Sriharsha Chintalapani wrote: config.originals() contains the user configured properties only it doesn't have defaults defined in ProducerConfig. On the side note shouldn't we use config.values() everywhere? Since I recently worked on KafkaConfig a bit, I can share my insights: Basically, the idea in passing properties along to pluggable tools is that we can define properties in our configuration files that Kafka doesn't know about (i.e. that are not declared with ConfigDef). Those will be defined by users, so originals() will include them. If a default exists, it means that Kafka knows about this configuration so you can pass the specific configuration you need along (in this case, SecurityProtocol). Note that this is all for pluggables - serializers, reporters, authorizers - which will not depend on Kafka and therefore can't reuse AbstractConfiguration classes. Channels will be implemented as part of Kafka and therefore you can probably just pass an AbstractConfiguration and use that. Hope this helps. - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84232 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 294 https://reviews.apache.org/r/33620/diff/8/?file=966806#file966806line294 Not sure why we are looping back here. If the HandshakeStatus is NEED_UNWRAP, we loop back w/o reading more data from the socket, will we get into infinite loop? answered below - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84232 --- On June 23, 2015, 8:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated June 23, 2015, 8:18 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. Diffs - build.gradle 30d1cf2f1ff9ed3f86a060da8099bb0774b4cf91 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 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 daff34db5bf2144e9dc274b23dc56b88f4efafdc clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 951c34c92710fc4b38d656e99d2a41255c60aeb7 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 5a37580ec69af08b97cf5b43b241790ba8c129dd 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/Channel.java PRE-CREATION 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/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/SSLFactory.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
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 June 4, 2015, 1:52 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. Diffs (updated) - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/ByteBufferReceive.java 129ae827bccbd982ad93d56e46c6f5c46f147fe0 clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java c8213e156ec9c9af49ee09f5238492318516aaa3 clients/src/main/java/org/apache/kafka/common/network/Channel.java PRE-CREATION 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/NetworkReceive.java fc0d168324aaebb97065b0aafbd547a1994d76a7 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/Receive.java 4e33078c1eec834bd74aabcb5fc69f18c9d6d52a clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/Send.java 5d321a09e470166a1c33639cf0cab26a3bce98ec clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java f3f8334f848be4cc043d5a573975609a3681fe7e clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review86250 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment138173 we also check handshakeStatus in other methods like handshakeWrap and handshakeUnwrap , handshakeFinished. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment138174 the reason we need to loop back during unwrap is run need_task case where the client need to check servers certificate or calculate keys etc.. The state will change once the tasks are done we won't be getting into infinite loop here. - Sriharsha Chintalapani On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 21, 2015, 5:37 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 240-242 https://reviews.apache.org/r/33620/diff/8/?file=966806#file966806line240 Would it be better to just throw the exception in the NOT_HANDSHAKING case in handshake()? we check handshakestatus for NOT_HANDSHAKING. During the ssl handshake this status can move from FINISHED to NOT_HANDSHAKING but that doesn't mean handshake is failed. Thats the reason I am checking both handshakeStatus and handshakeResult if the handshake is finsihed or not. On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 343 https://reviews.apache.org/r/33620/diff/8/?file=966806#file966806line343 Hmm, shoudn't we check for remaining() 0 here? we always call appReadBuffer.compact in method readAppBuffer this will reset the appReadBuffer.remaining() to the capacity so checking for that won't work and position() gives if we have more data or not in this case. On May 22, 2015, 12:33 a.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 388 https://reviews.apache.org/r/33620/diff/8/?file=966806#file966806line388 Not sure why we need the loop here. It seems that netReadBuffer.position is always 0 after the compact in line 357. If we are unable to unwrap all the netReadBuffer than the position in netReadBuffer won't be 0 after the compact. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84232 --- On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 21, 2015, 5:37 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
- Michael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 21, 2015, 5:37 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/33620/diff/ Testing --- Thanks, Sriharsha Chintalapani
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 10:54 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 153 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line153 I think Michael meant the following which I think is valid right? Line 146: handshakeWrap completes and the status changes to NEED_UNWRAP (since SSLEngine has finished wrapping), but the netBuffer has not yet been flushed. So on line 153 we would fall through to the NEED_UNWRAP case without doing the flush. Joel Koshy wrote: Discussed offline. I misread the code. This logic is fine. Michael, let us know if you still think there is an issue. I misread the code, too. The only issue I see is that two readers misread the code in the same way ;) - Michael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 21, 2015, 5:37 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On May 22, 2015, 12:14 a.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java, line 44 https://reviews.apache.org/r/33620/diff/8/?file=966813#file966813line44 I'm trying to imagine implementing `buildPrincipal` in such a way as to make a Principal from information found in the peer's certifcate: how could I obtain that? I don't see (immediately) a way to navigate from either the `TransportLayer` or the `Authenticator` to the current `SSLSession`... Sriharsha Chintalapani wrote: The reason I added bothe TransportLayer and Authenticator to the buildPrincipal to give user option to use one or both of them to construct a principal. It depends on the which TransportLayer or Authenticator they might be using. in the case you pointed out, one can do this in buildPrincipal if (transportLayer is instanceOf SSLTransportLayer) SSLSession = ((SSLTransportLayer) sslTransportLayer).sslSession(); Got it-- thx. - Michael --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84235 --- On May 21, 2015, 5:37 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 21, 2015, 5:37 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION
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 May 21, 2015, 5:37 p.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. Diffs (updated) - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/33620/diff/ Testing --- Thanks, Sriharsha Chintalapani
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java, lines 137-139 https://reviews.apache.org/r/33620/diff/5/?file=957064#file957064line137 This is interesting; I don't see a corresponding `createSSLSocketFactory`-- did I miss it? Besides, I thought Kafka used NIO (in which case you wouldn't be using `SSLServerSocketFactory`, which IIRC is blocking. Who calls this? This method is used for tests and I moved it under tests. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 56-59 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line56 Do you want to make your nomenclature more uniform? The network buffers are referred to in terms of in out while the application buffers are referred to as read write. On a related note, do you need read write for each? At any given point in time, you're doing one or the other, so why not just have a network buffer an application buffer? Even though we do one op at any given time we still want two buffer as we might not have read the entire buffer in op invocation and we need to keep this data around. Having two buffers for read and write makes it easier. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 76 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line76 I'm confused on the difference between `startHandshake` `handshake` (inherited from interface `TransportLayer`). Sriharsha Chintalapani wrote: startHandshake is more of a prepare method for handshake process and its only SSLTransportLayer and I'll make that as a private method. handshake method is for general TransportLayer handshake . As indicated in comments for PlainTextChannel its a non-op and for SSL its implemented as non-blocking ssl handshake. I made startHandshake to private and its prepare method when the transportLayer gets created. If you see the naming of the method confusing please re-open. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83807 --- On May 20, 2015, 9:54 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 20, 2015, 9:54 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. Diffs - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java PRE-CREATION
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 May 20, 2015, 9:54 p.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. Diffs (updated) - build.gradle cd2aa838fd53e8124f308979b1d70efe0c5725a6 checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/33620/diff/ Testing --- Thanks, Sriharsha Chintalapani
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 418 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line418 So you're transferring bytes from `appReadBuffer` to `dst`, right? Does not `ByteBuffer.put` do this for you? Sriharsha Chintalapani wrote: acutally ByteBuffer.put doesn't know how much data that source buffer has and how much remaining and also it doesn't cover the case where the dst buffer might not have enough space. I can do it like this , but it needs additional tmp buffer. private int readFromAppBuffer(ByteBuffer dst) { appReadBuffer.flip(); int remaining = Math.min(dst.remaining(), appReadBuffer.remaining()); if (remaining 0) { ByteBuffer tmp = appReadBuffer.duplicate(); tmp.limit(tmp.position() + remaining); dst.put(tmp); appReadBuffer.position(appReadBuffer.position() + remaining); } appReadBuffer.compact(); return remaining; } ignore the comment above. I fixed it. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83807 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 112-121 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line112 Sorry, but I'm a little confused here. Why would the caller be passing `read` `write`? I mean, you own the underlying socket, so you know that state it's in. And you also have the handshake status, so you know what you're expecting, right? Update: I've stepped through your logic, and it seems to me that you're figuring out at the end of each pass whether you want to read or write, returning that information to the caller, and asking the caller to pass it back in the next invocation of `handshake`. Why? Why not just record your state in a member variable? Sriharsha Chintalapani wrote: I am depending on the SelectionKey interest Ops. Socketchannel http://docs.oracle.com/javase/7/docs/api/java/nio/channels/SocketChannel.html doesn't tell me if its ready for write or read. We depend on the Selector's selectionKey ops to figure out if the socketChannel is ready for write or read ops. I wanted to manage the selectionKey ops in the Selector not in transportLayer. The new patch that I am going to send this will change this. Again why do you think this an issue. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 418 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line418 So you're transferring bytes from `appReadBuffer` to `dst`, right? Does not `ByteBuffer.put` do this for you? acutally ByteBuffer.put doesn't know how much data that source buffer has and how much remaining and also it doesn't cover the case where the dst buffer might not have enough space. I can do it like this , but it needs additional tmp buffer. private int readFromAppBuffer(ByteBuffer dst) { appReadBuffer.flip(); int remaining = Math.min(dst.remaining(), appReadBuffer.remaining()); if (remaining 0) { ByteBuffer tmp = appReadBuffer.duplicate(); tmp.limit(tmp.position() + remaining); dst.put(tmp); appReadBuffer.position(appReadBuffer.position() + remaining); } appReadBuffer.compact(); return remaining; } - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83807 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84144 --- clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.java https://reviews.apache.org/r/33620/#comment135274 Thank you for adding this option. Maybe the default should be null so that it is possible to disable endpoint verification? - Rajini Sivaram On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/33620/diff/ Testing ---
Re: Review Request 33620: Patch for KAFKA-1690
On May 14, 2015, 10:21 a.m., Rajini Sivaram wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 190 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line190 I think when delegated tasks are run asynchronously, selection interestOps should be managed within the transport layer. Otherwise, packets may be processed out of order. InterestOps should be set to zero before tasks are executed, and changes to interestOps due to packets being ready for transmission etc. should be cached and set after the delegated tasks are executed. Not sure if the current transport layer interface is sufficient to handle this correctly. Sriharsha Chintalapani wrote: I am not sure how packets can be processed out of order here. If the handshakeStatus is NEED_TASK I call tasks() method which hands over any delegated tasks to executorService . As long as these tasks are not finished handshakeStatus stays NEED_TASK only after these tasks are finished we are resuming handshake. There is no further reading/writing to socketChannel until these tasks are finished. Can you elaborate on why you need InterestOps to set to 0. I think what happens at the moment is that handshake with a delegated task doesn't change interestOps. So it results in a tight loop when interestOps has write enabled. In this case, tasks() is called over and over again when poll() returns, until the delegated task is complete. This is undesirable, but doesn't lead to errors. In the case of delegated tasks in read/write, I am not sure the code checks for NEED_TASK upfront, so it is not clear if they dont lead to packets being processed out of order because of interestOps not being reset. Basically we want to avoid processing any packets while potentially long running delegated tasks are being processed on a different thread. It feels like this is being achieved with a tight polling loop at the moment, which negates the benefit of running these tasks in a different thread. Setting interestOps to zero and resetting them when the task is complete would avoid the polling overhead and simplify the code in read/write which otherwise need to check if another thread is handling a delegated task. On May 14, 2015, 10:21 a.m., Rajini Sivaram wrote: clients/src/main/java/org/apache/kafka/common/network/Authenticator.java, line 47 https://reviews.apache.org/r/33620/diff/5/?file=957060#file957060line47 Don't think authenticator interface should handle selection interestOps. Needs better encapsulation. Sriharsha Chintalapani wrote: SaslAuth works by reading socketChannel and doing accept token on incoming data on server side and similarly saslclient sends a new token . Why do you think authenticate shouldn't handle interestOps since its reading and writing to socketChannel. Whats your proposal Rajini Sivaram wrote: At the moment management of interestOps is spread across Selector, TransportLayer and Authenticator. I would have hoped that they could be contained within the transport layer. My concern is that with transport layer setting interestOps from multiple threads (for delegated tasks in SSL), it may be too messy to handle interestOps within a pluggable authenticator as well. Maybe it is better to address this when SaslAuth implementation is ready. Sriharsha Chintalapani wrote: Only selector is the one setting the interestOps not transportLayer or authenticator , they just return a interstOp. How is multiple threads an issue here? May not be an issue if re-negotiation and full graceful shutdown are not required. Will wait to see how interestOps are handled for delegated tasks (in the other issue raised) before closing this one. - Rajini --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83741 --- On May 12, 2015, 11:20 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 12, 2015, 11:20 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84012 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135138 Sriharsha, this is the line from the ref guide under blocking tasks I was referring to: The engine will block future wrap/unwrap calls until all of the outstanding tasks are completed - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff:
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java, line 34 https://reviews.apache.org/r/33620/diff/5/?file=957068#file957068line34 This is probably just my ignorance of the Kafka codebase, but what does this interface represent? It looks a lot like a socket to me... It does represet a socket but instead of extending its acting as delegation layer to the socket and allowing me to read/write methods without extending socketchannel . Do you see this as a issue? On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 404-410 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line404 This is likely to be unhelpful in many cases; according to the javadoc for class X500Principal, the name will be derived from the Subject Name in the peer certificate. Typically, the Subject Name is the hostname of the cert to permit peer validation when tools like curl or a browser are used to hit the endpoint (for debugging purposes, say). At LinkedIn, we use the Subject Alternative Names field to embed service identity. In the latest version there is a PrincipalBuilder class which is pluggable which takes in both authenticator and transportLayer. Its up to user which one they want to use. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 251 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line251 I'm not familiar with Kafka's threading model, but I know throughput is very important to you. Do you want to be blocking execution on this thread while you carry out these tasks? tasks() method runs the delegatedTasks using ExecutorService so it won't run on selector thread. But Rajini brought up an issue about muting the selectionKey I am working on it. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 215 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line215 Did you remember to flip `appWriteBuffer` before calling `wrap`? not necessary here. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 153 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line153 I don't know if this is right. Suppose we return from `handshakeWrap` with data still to be written from `netOutBuffer` to the socket. The status will still be NEEDS_UNWRAP, because as far as SSLEngine is concerned, it's wrapped everything ready for input. In this case, you'll fall through without flushing everything from `netOutBuffer`. Not sure if I understood the comment. Suppose we return from handshakeWrap with data still to be written from netOutBuffer to the socket. If we still have data in netOutBuffer we will return from handshake process with WRITE selectionOp.Interest to register for the next iteration. we don't fall through without flushing everyting from netOutBuffer. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, lines 112-121 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line112 Sorry, but I'm a little confused here. Why would the caller be passing `read` `write`? I mean, you own the underlying socket, so you know that state it's in. And you also have the handshake status, so you know what you're expecting, right? Update: I've stepped through your logic, and it seems to me that you're figuring out at the end of each pass whether you want to read or write, returning that information to the caller, and asking the caller to pass it back in the next invocation of `handshake`. Why? Why not just record your state in a member variable? I am depending on the SelectionKey interest Ops. Socketchannel http://docs.oracle.com/javase/7/docs/api/java/nio/channels/SocketChannel.html doesn't tell me if its ready for write or read. We depend on the Selector's selectionKey ops to figure out if the socketChannel is ready for write or read ops. On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 76 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line76 I'm confused on the difference between `startHandshake` `handshake` (inherited from interface `TransportLayer`). startHandshake is more of a prepare method for handshake process and its only SSLTransportLayer and I'll make that as a private method. handshake method is for general TransportLayer handshake . As indicated in comments for
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java, line 29 https://reviews.apache.org/r/33620/diff/5/?file=957072#file957072line29 Will it be pluggable; i.e. can individual sites provide their own implementations? This is made it into plugabble class. Please check the revision 6 of the patch. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83807 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135093 I think Michael meant the following which I think is valid right? Line 146: handshakeWrap completes and the status changes to NEED_UNWRAP (since SSLEngine has finished wrapping), but the netBuffer has not yet been flushed. So on line 153 we would fall through to the NEED_UNWRAP case without doing the flush. clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135095 Actually, can you describe how this would be done (say, for dealing with revoked certificates after an client authenticates)? Per your jira comment we can use an authorizer to block the client in this case, but if you have a proposal on handling periodic renegotiation it would be useful to discuss that. I agree we don't need to implement it now. - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review84006 --- clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135132 From the ref guide: In addition to an application explicitly closing the SSLEngine, the SSLEngine might be closed by the peer (via receipt of a close message while it is processing handshake data), or by the SSLEngine encountering an error while processing application or handshake data, indicated by throwing an SSLException. In such cases, the application should invoke SSLEngine.wrap() to get the close message and send it to the peer until SSLEngine.isOutboundDone() returns true, as shown in the previous example, or the SSLEngineResult.getStatus() returns CLOSED. Do we need to do this? i.e., it appears we are not handling this during the handshake. - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
On May 15, 2015, 10:54 p.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, line 153 https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line153 I think Michael meant the following which I think is valid right? Line 146: handshakeWrap completes and the status changes to NEED_UNWRAP (since SSLEngine has finished wrapping), but the netBuffer has not yet been flushed. So on line 153 we would fall through to the NEED_UNWRAP case without doing the flush. Discussed offline. I misread the code. This logic is fine. Michael, let us know if you still think there is an issue. - Joel --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83993 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION
Re: Review Request 33620: Patch for KAFKA-1690
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83998 --- Few more minor comments. General comment on javadocs: although this can wait. The current patch has some but hopefully the final patch will have more uniformly thorough javadocs. Similar comment on logging. E.g., should we add trace messages in various stages of the handshake? BTW thanks a lot for the patch - this is a lot of work! clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java https://reviews.apache.org/r/33620/#comment135098 May be able to remove values() if we avoid passing in the full config as mentioned in Selector clients/src/main/java/org/apache/kafka/common/network/Authenticator.java https://reviews.apache.org/r/33620/#comment135119 I'm also unclear on this API. Will discuss offline with you. clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java https://reviews.apache.org/r/33620/#comment135105 These can be private. clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java https://reviews.apache.org/r/33620/#comment135121 May want to just have a final member and return that. clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java https://reviews.apache.org/r/33620/#comment135106 Can we just use a fixed thread pool? A more general question/concern though: If there are multiple inbound connections all of which end up needing delegated tasks, wouldn't they block? i.e., since the pool size is one? From my reading of the JSSE ref guide this would typically be when you have a custom trust or key manager. Are there other use-cases? clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135114 `if (handshakeResult.getStatus() == Status.OK handshakeStatus == HandshakeStatus.NEED_TASK)` clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java https://reviews.apache.org/r/33620/#comment135113 Should this be unwrapped into the appReadBuffer? clients/src/main/java/org/apache/kafka/common/network/Selector.java https://reviews.apache.org/r/33620/#comment135100 Rather than leak in the entire config here, should we just initialize SSLFactory outside and pass that in instead? We can add constructors for different authentication mechanisms as needed. clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java https://reviews.apache.org/r/33620/#comment135115 typo clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java https://reviews.apache.org/r/33620/#comment135116 1, SSL clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java https://reviews.apache.org/r/33620/#comment135117 @override annotations on this and couple other methods. - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5
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 May 15, 2015, 2:18 p.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. Diffs (updated) - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java f73eedb030987f018d8446bb1dcd98d19fa97331 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java d5b306b026e788b4e5479f3419805aa49ae889f3 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 2ebe3c21f611dc133a2dbb8c7dfb0845f8c21498 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION Diff: https://reviews.apache.org/r/33620/diff/ Testing --- Thanks, Sriharsha Chintalapani
Re: Review Request 33620: Patch for KAFKA-1690
On May 13, 2015, 3:40 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/common/network/Channel.java, lines 70-71 https://reviews.apache.org/r/33620/diff/5/?file=957061#file957061line70 This is a general question. For ssl and sasl, does authentication only happen at connection time for ssl and sasl? What happens when a certificate/credential is removed? Do we have to re-authenticate on the existing connection? Addressed this on JIRA comments. On May 13, 2015, 3:40 p.m., Jun Rao wrote: clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java, lines 287-289 https://reviews.apache.org/r/33620/diff/5/?file=957055#file957055line287 Should we give those fields a default? The user only needs to configure them if ssl is enabled. right now we are checking for obsense of these on client side to configure SSLFactory. If we give defaults than client side will throw an exception. On May 13, 2015, 3:40 p.m., Jun Rao wrote: clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java, line 39 https://reviews.apache.org/r/33620/diff/4/?file=955788#file955788line39 Does that bind the port? If so, we will need to select a random port. It doesn't bind to a port. Its a hint to SSLEngine. - Sriharsha --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83322 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f
Re: Review Request 33620: Patch for KAFKA-1690
On May 16, 2015, 12:07 a.m., Joel Koshy wrote: clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java, line 35 https://reviews.apache.org/r/33620/diff/6/?file=961241#file961241line35 Can we just use a fixed thread pool? A more general question/concern though: If there are multiple inbound connections all of which end up needing delegated tasks, wouldn't they block? i.e., since the pool size is one? From my reading of the JSSE ref guide this would typically be when you have a custom trust or key manager. Are there other use-cases? Capturing offline discussion: I'm actually less concerned about this on the client side than the server side. On the client side the net effect of blocking here is that the producer/consumer would be to not produce/consume during the handshake. On the current server-side current patch (KAFKA-1684) it is done in the acceptor threads. So on a server bounce several clients could initiate handshakes on new partition leaders. If there are several delegated tasks then all processors could eventually block. Not sure if this is something we need to worry about. I wonder why SSLEngine does not expose a callback-based model for executing delegated tasks. - Joel --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/#review83998 --- On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 15, 2015, 2:18 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. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b checkstyle/checkstyle.xml a215ff36e9252879f1e0be5a86fef9a875bb8f38 checkstyle/import-control.xml f2e6cec267e67ce8e261341e373718e14a8e8e03 clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 0d68bf1e1e90fe9d5d4397ddf817b9a9af8d9f7a clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java cf32e4e7c40738fe6d8adc36ae0cfad459ac5b0b clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java bdff518b732105823058e6182f445248b45dc388 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java d301be4709f7b112e1f3a39f3c04cfa65f00fa60 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 8e336a3aa96c73f52beaeb56b931baf4b026cf21 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 187d0004c8c46b6664ddaffecc6166d4b47351e5 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java c4fa058692f50abb4f47bd344119d805c60123f5 clients/src/main/java/org/apache/kafka/common/config/SecurityConfigs.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/Channel.java PRE-CREATION 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/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/SSLFactory.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 b5f8d83e89f9026dc0853e5f92c00b2d7f043e22 clients/src/main/java/org/apache/kafka/common/network/Selector.java 57de0585e5e9a53eb9dcd99cac1ab3eb2086a302 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java dab1a94dd29563688b6ecf4eeb0e180b06049d3f