> 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 
>   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 
>   
> 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 
> af9993cf9b3991f1e61e1201c94e19bc1bf76a68 
>   clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java 
> 13ce519f03d13db041e1f2dbcd6b59414d2775b8 
>   
> 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 
>   clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java 
> PRE-CREATION 
>   clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 
> 158f9829ff64a969008f699e40c51e918287859e 
>   clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 
> e7951d835472e5defe49be435f2c93685ba544d5 
>   clients/src/test/java/org/apache/kafka/test/MockSelector.java 
> 51eb9d142f566c94a87add68b8c4f78b56d6ec3e 
>   clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java PRE-CREATION 
>   core/src/main/scala/kafka/network/SocketServer.scala 
> 91319fa010b140cca632e5fa8050509bd2295fc9 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 
> dbe170f87331f43e2dc30165080d2cb7dfe5fdbf 
>   core/src/main/scala/kafka/server/KafkaServer.scala 
> 18917bc4464b9403b16d85d20c3fd4c24893d1d3 
>   core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 
> 9ce4bd5ee130ce3cb252b2883a3fd3c9acd742a5 
>   core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala 
> PRE-CREATION 
>   core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala 
> PRE-CREATION 
>   core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala 
> 8b14bcfe7af601fe4b0fb0a7c0c544e87403062a 
>   core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala 
> e4bf2df48dd59a251b646b7f96d63ec4b924fc0b 
>   core/src/test/scala/unit/kafka/network/SocketServerTest.scala 
> 7dc2fad542ea553ee888543dd215eb41ea57d509 
>   core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 
> 04a02e08a54139ee1a298c5354731bae009efef3 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> eb169d8b33c27d598cc24e5a2e5f78b789fa38d3 
> 
> Diff: https://reviews.apache.org/r/33620/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>

Reply via email to