-----------------------------------------------------------
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 
> 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
> 
>

Reply via email to