> 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 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 
>   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 
>   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/security/ssl/SSLFactory.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/common/utils/Utils.java 
> 80a914ee1d12a4b855fbdab24c7e870d33fb475e 
>   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/SSLSelectorTest.java 
> PRE-CREATION 
>   clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 
> 158f9829ff64a969008f699e40c51e918287859e 
>   
> clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java
>  PRE-CREATION 
>   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/api/FetchResponse.scala 
> 0b6b33ab6f7a732ff1322b6f48abd4c43e0d7215 
>   core/src/main/scala/kafka/network/SocketServer.scala 
> dbe784b63817fd94e1593136926db17fac6fa3d7 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 
> 394f21b5d70bd527e77b611b2d1f9b36f0d0be81 
>   core/src/main/scala/kafka/server/KafkaServer.scala 
> 6d6550732f393037ed1750cc8a7010a29d2c574a 
>   core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 
> 4b6358c2329448a3b3fbdce455c7f38b3978a274 
>   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 
> 05b9a87d6918ba93cb2ab364ae7bd52f8d9b4b78 
>   core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala 
> 4dba7dce57e674104f34556018fa8f1543c6c247 
>   core/src/test/scala/unit/kafka/network/SocketServerTest.scala 
> d08b8b833110771a2a33ce73f09b62e445f96a0c 
>   core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala 
> 9688b8cf64d3d74021d21eb516ebd6ba424e013c 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> f00f00a260af4b0bc503e1f9096749d7d4ad1f24 
> 
> Diff: https://reviews.apache.org/r/33620/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Sriharsha Chintalapani
> 
>

Reply via email to