[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14984140#comment-14984140 ] Jeffrey Turpin commented on SPARK-6373: --- [~jlewandowski], are you willing to do a review of my changes? I can create a pull request if you would like? > Add SSL/TLS for the Netty based BlockTransferService > - > > Key: SPARK-6373 > URL: https://issues.apache.org/jira/browse/SPARK-6373 > Project: Spark > Issue Type: New Feature > Components: Block Manager, Shuffle >Affects Versions: 1.2.1 >Reporter: Jeffrey Turpin > > Add the ability to allow for secure communications (SSL/TLS) for the Netty > based BlockTransferService and the ExternalShuffleClient. This ticket will > hopefully start the conversation around potential designs... Below is a > reference to a WIP prototype which implements this functionality > (prototype)... I have attempted to disrupt as little code as possible and > tried to follow the current code structure (for the most part) in the areas I > modified. I also studied how Hadoop achieves encrypted shuffle > (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) > https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14984219#comment-14984219 ] Jeffrey Turpin commented on SPARK-6373: --- Any comments/feedback would be appreciated... https://github.com/turp1twin/spark/commit/fd2980ab8cc1fc5b4626bb7a0d1e94128ca3874d > Add SSL/TLS for the Netty based BlockTransferService > - > > Key: SPARK-6373 > URL: https://issues.apache.org/jira/browse/SPARK-6373 > Project: Spark > Issue Type: New Feature > Components: Block Manager, Shuffle >Affects Versions: 1.2.1 >Reporter: Jeffrey Turpin > > Add the ability to allow for secure communications (SSL/TLS) for the Netty > based BlockTransferService and the ExternalShuffleClient. This ticket will > hopefully start the conversation around potential designs... Below is a > reference to a WIP prototype which implements this functionality > (prototype)... I have attempted to disrupt as little code as possible and > tried to follow the current code structure (for the most part) in the areas I > modified. I also studied how Hadoop achieves encrypted shuffle > (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) > https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14980657#comment-14980657 ] Jeffrey Turpin commented on SPARK-6373: --- [~jlewandowski], Yes I am. I got crazy busy at work the last few months and I had to table this work. I wasn't getting much feedback, so any feedback you want to give would be appreciated. I have a couple small refactorings to finish and then I will push my latest to my branch, and can create a PR after that... Sorry for the long delays... Jeff > Add SSL/TLS for the Netty based BlockTransferService > - > > Key: SPARK-6373 > URL: https://issues.apache.org/jira/browse/SPARK-6373 > Project: Spark > Issue Type: New Feature > Components: Block Manager, Shuffle >Affects Versions: 1.2.1 >Reporter: Jeffrey Turpin > > Add the ability to allow for secure communications (SSL/TLS) for the Netty > based BlockTransferService and the ExternalShuffleClient. This ticket will > hopefully start the conversation around potential designs... Below is a > reference to a WIP prototype which implements this functionality > (prototype)... I have attempted to disrupt as little code as possible and > tried to follow the current code structure (for the most part) in the areas I > modified. I also studied how Hadoop achieves encrypted shuffle > (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) > https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6229) Support SASL encryption in network/common module
[ https://issues.apache.org/jira/browse/SPARK-6229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14487878#comment-14487878 ] Jeffrey Turpin commented on SPARK-6229: --- Hey Marcelo, I have been working on SPARK-6373 and have reviewed you pull request and merged into my wip https://github.com/turp1twin/spark/tree/ssl-shuffle. I tried to follow the general design pattern that I discussed with Aaron Davidson, by having a single EncryptionHandler interface and implementations for both SSL and SASL Encryption. One issue I faced is that the timing of adding the appropriate encryption handlers differs for SSL and SASL. For SSL, I need to add the SslHandler to the Netty pipeline before the connection is made, and for SASL, it looks like you add it during the TransportClient/Server Bootstrap process which occurs after the initial connection. Anyways, I haven't created a pull request yet and am waiting on some more feedback... If you have some time perhaps you can give me your thoughts... Some commits of interest... https://github.com/apache/spark/commit/ab8743f6ac707060cbae63bdf491723709fe32f3 https://github.com/apache/spark/commit/9527aef89b1bbc80a22337552dd54af936aa1094 Cheers, Jeff Support SASL encryption in network/common module Key: SPARK-6229 URL: https://issues.apache.org/jira/browse/SPARK-6229 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Marcelo Vanzin After SASL support has been added to network/common, supporting encryption should be rather simple. Encryption is supported for DIGEST-MD5 and GSSAPI. Since the latter requires a valid kerberos login to work (and so doesn't really work with executors), encryption would require the use of DIGEST-MD5. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6229) Support SASL encryption in network/common module
[ https://issues.apache.org/jira/browse/SPARK-6229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14488104#comment-14488104 ] Jeffrey Turpin commented on SPARK-6229: --- Hey Marcelo, So what I have done is to overload the TransportContext constructor, adding a constructor that takes an instance of the TransportEncryptionHandler interface: {code:title=TransportContext.java|linenumbers=false|language=java} public TransportContext( TransportConf conf, RpcHandler appRpcHandler, TransportEncryptionHandler encryptionHandler) { this.conf = conf; this.appRpcHandler = appRpcHandler; this.decoder = new MessageDecoder(); if (encryptionHandler != null) { this.encryptionHandler = encryptionHandler; } else { this.encryptionHandler = new NoEncryptionHandler(); } this.encoder = (this.encryptionHandler.isEnabled() ? new SslMessageEncoder() : new MessageEncoder()); } {code} This way the method existing method signatures for createServer and createClientFactory don't change. To facilitate this I also added a constructor to the TransportClientFactory class and modified the constructor for the TransportServer class, to also take a TransportEncryptionHandler instance In the TransportClientFactory case I need to add the Netty SslHandler before the connection occurs, which can be done by calling the _addToPipeline_ method of the TransportEncryptionHandler interface: {code:title=TransportClientFactory.java|linenumbers=false|language=java} private void initHandler( final Bootstrap bootstrap, final AtomicReferenceTransportClient clientRef, final AtomicReferenceChannel channelRef) { bootstrap.handler(new ChannelInitializerSocketChannel() { @Override protected void initChannel(SocketChannel ch) throws Exception { TransportChannelHandler clientHandler = context.initializePipeline(ch); encryptionHandler.addToPipeline(ch.pipeline(), true); clientRef.set(clientHandler.getClient()); channelRef.set(ch); } }); } {code} This _initHandler_ method is called just before connection is made. In addition the TransportEncryptionHandler interface has an _onConnect_ method to allow a post connect initialization to occur, which in the SSL case, is to allow the handshake process to complete, which is a blocking operation. This could be possibly done in a custom TransportClientBootstrap implementation, but the method signature of _doBootstrap_ would have to change to allow for this. As for the TransportServer, the Netty SslHandler must be added to the pipeline before the server binds to a port and starts listening for connections. Again, in this case, this could be done in a TransportServerBootstrap implementation, but the method signature of _doBootstrap_ would have to change (or we would need to add another method) to allow for this... Thoughts? Jeff Support SASL encryption in network/common module Key: SPARK-6229 URL: https://issues.apache.org/jira/browse/SPARK-6229 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Marcelo Vanzin After SASL support has been added to network/common, supporting encryption should be rather simple. Encryption is supported for DIGEST-MD5 and GSSAPI. Since the latter requires a valid kerberos login to work (and so doesn't really work with executors), encryption would require the use of DIGEST-MD5. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14391452#comment-14391452 ] Jeffrey Turpin commented on SPARK-6373: --- Hey Aaron, Sorry for the delay... I have cleaned things up a bit and refactored the implementation to be more inline with our earlier conversation... Have a look at https://github.com/turp1twin/spark/commit/d976a7ab9b57e26fc180d649fd084a6acb9d027e and let me know your thoughts... Jeff Add SSL/TLS for the Netty based BlockTransferService - Key: SPARK-6373 URL: https://issues.apache.org/jira/browse/SPARK-6373 Project: Spark Issue Type: New Feature Components: Block Manager, Shuffle Affects Versions: 1.2.1 Reporter: Jeffrey Turpin Add the ability to allow for secure communications (SSL/TLS) for the Netty based BlockTransferService and the ExternalShuffleClient. This ticket will hopefully start the conversation around potential designs... Below is a reference to a WIP prototype which implements this functionality (prototype)... I have attempted to disrupt as little code as possible and tried to follow the current code structure (for the most part) in the areas I modified. I also studied how Hadoop achieves encrypted shuffle (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14378375#comment-14378375 ] Jeffrey Turpin commented on SPARK-6373: --- H... I am using Intellij, perhaps it messed the formatting up... I will check I will get a patch ready for review Thanks! Jeff Add SSL/TLS for the Netty based BlockTransferService - Key: SPARK-6373 URL: https://issues.apache.org/jira/browse/SPARK-6373 Project: Spark Issue Type: New Feature Components: Block Manager, Shuffle Affects Versions: 1.2.1 Reporter: Jeffrey Turpin Add the ability to allow for secure communications (SSL/TLS) for the Netty based BlockTransferService and the ExternalShuffleClient. This ticket will hopefully start the conversation around potential designs... Below is a reference to a WIP prototype which implements this functionality (prototype)... I have attempted to disrupt as little code as possible and tried to follow the current code structure (for the most part) in the areas I modified. I also studied how Hadoop achieves encrypted shuffle (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14376907#comment-14376907 ] Jeffrey Turpin commented on SPARK-6373: --- Hey Aaron, Thanks for the feedback! I definitely agree we should find a common way to support both and your proposal sounds good to me. That being said, do you want me to take a cut at doing this, integrating the work I have already done (where applicable)? Definitely don't want to lose traction on this and would like to get it into a release sooner rather than later Let me know your thoughts... Cheers! Jeff Add SSL/TLS for the Netty based BlockTransferService - Key: SPARK-6373 URL: https://issues.apache.org/jira/browse/SPARK-6373 Project: Spark Issue Type: New Feature Components: Block Manager, Shuffle Affects Versions: 1.2.1 Reporter: Jeffrey Turpin Add the ability to allow for secure communications (SSL/TLS) for the Netty based BlockTransferService and the ExternalShuffleClient. This ticket will hopefully start the conversation around potential designs... Below is a reference to a WIP prototype which implements this functionality (prototype)... I have attempted to disrupt as little code as possible and tried to follow the current code structure (for the most part) in the areas I modified. I also studied how Hadoop achieves encrypted shuffle (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14372003#comment-14372003 ] Jeffrey Turpin commented on SPARK-6373: --- Here are some comments about my implementation... *Configuration:* I added a new SSLOptions member variable to SecurityManager.scala, specifically for configuring SSL for the Block Transfer Service: {code:title=SecurityManager.scala|linenumbers=false|language=scala} val btsSSLOptions = SSLOptions.parse(sparkConf, spark.ssl.bts, Some(defaultSSLOptions)) {code} I expanded the SSLOptions case class to capture additional SSL related parameters: {code:title=SecurityManager.scala|linenumbers=false|language=scala} private[spark] case class SSLOptions( enabled: Boolean = false, keyStore: Option[File] = None, keyStorePassword: Option[String] = None, privateKey: Option[File] = None, keyPassword: Option[String] = None, certChain: Option[File] = None, trustStore: Option[File] = None, trustStorePassword: Option[String] = None, trustStoreReloadingEnabled: Boolean = false, trustStoreReloadInterval: Int = 1, openSslEnabled: Boolean = false, protocol: Option[String] = None, enabledAlgorithms: Set[String] = Set.empty) {code} This can change, but for now I added the ability to provide a standard java keystore and truststore, as was possible with the existing file server and akka SSL configurations available in SecurityManager.scala. For this configuration I added the ability to enable truststore reloading (hadoop encrypted shuffle allows for this). In addition, I added the ability to specify an X.509 certificate chain in PEM format and a PKCS#8 private key file in PEM format. If all four parameters are provided (keyStore, trustStore, privateKey, certChain) then the privateKey and certChain parameters will be used. In TransportConf.java I added two addition configuration parameters: {code:title=TransportConf.java|linenumbers=false|language=java} public int sslShuffleChunkSize() { return conf.getInt(spark.shuffle.io.ssl.chunkSize, 60 * 1024); } public boolean sslShuffleEnabled() { return conf.getBoolean(spark.ssl.bts.enabled, false); } {code} For the _spark.shuffle.io.ssl.chunkSize_ config param I set the default to the same size used in Hadoop's encrypted shuffle implementation. *Implementation:* For this prototype, I opted to disrupt as little code as possible, meaning I wanted to avoid any major refactorings... So here is some of what I did... I essentially overloaded two methods in TransportContext.java which take a new class I created, SSLFactory. SSLFactory basically encapsulates all the Keystore/Certificate stuff that needs to happen. {code:title=TransportContext.java|linenumbers=false|language=java} public TransportClientFactory createClientFactory( ListTransportClientBootstrap bootstraps, SSLFactory sslFactory) { return new TransportClientFactory(this, bootstraps, sslFactory); } ... public TransportServer createSecureServer(SSLFactory sslFactory) { return new TransportServer(this, 0, sslFactory); } public TransportServer createSecureServer(SSLFactory sslFactory, int port) { return new TransportServer(this, port, sslFactory); } {code} These new methods all the user to create a secure TransportServer or secure TransportClientFactory instance. Obviously I had to add the appropriate constructors to both the TransportClientFactory and TransportServer classes Now in the _createClient_ method of TransportClientFactory, it will essentially check if the _sslFactory_ member variable is null, and if not, will initialize the Netty pipeline and add a Netty SslHandler... {code:title=TransportClientFactory.java|linenumbers=false|language=java} private void initHandler( final Bootstrap bootstrap, final AtomicReferenceTransportClient clientRef) { if (sslFactory != null) { bootstrap.handler(new ChannelInitializerSocketChannel() { @Override protected void initChannel(SocketChannel ch) throws Exception { ChannelPipeline pipeline = ch.pipeline(); // Add SSL handler first to encrypt and decrypt everything. pipeline.addLast(new SslHandler(sslFactory.createSSLEngine())); TransportChannelHandler clientHandler = context.initializePipeline(ch); clientRef.set(clientHandler.getClient()); } }); } else { bootstrap.handler(new ChannelInitializerSocketChannel() { @Override public void initChannel(SocketChannel ch) { TransportChannelHandler clientHandler = context.initializePipeline(ch); clientRef.set(clientHandler.getClient()); } }); } } {code} In addition, the _createClient_ method will block and wait for the SSL handshake to complete before returning a TransportClient instance... The TransportServer implementation
[jira] [Updated] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
[ https://issues.apache.org/jira/browse/SPARK-6373?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeffrey Turpin updated SPARK-6373: -- Priority: Major (was: Minor) Add SSL/TLS for the Netty based BlockTransferService - Key: SPARK-6373 URL: https://issues.apache.org/jira/browse/SPARK-6373 Project: Spark Issue Type: New Feature Components: Block Manager, Shuffle Affects Versions: 1.2.1 Reporter: Jeffrey Turpin Add the ability to allow for secure communications (SSL/TLS) for the Netty based BlockTransferService and the ExternalShuffleClient. This ticket will hopefully start the conversation around potential designs... Below is a reference to a WIP prototype which implements this functionality (prototype)... I have attempted to disrupt as little code as possible and tried to follow the current code structure (for the most part) in the areas I modified. I also studied how Hadoop achieves encrypted shuffle (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6373) Add SSL/TLS for the Netty based BlockTransferService
Jeffrey Turpin created SPARK-6373: - Summary: Add SSL/TLS for the Netty based BlockTransferService Key: SPARK-6373 URL: https://issues.apache.org/jira/browse/SPARK-6373 Project: Spark Issue Type: New Feature Components: Block Manager, Shuffle Affects Versions: 1.2.1 Reporter: Jeffrey Turpin Priority: Minor Add the ability to allow for secure communications (SSL/TLS) for the Netty based BlockTransferService and the ExternalShuffleClient. This ticket will hopefully start the conversation around potential designs... Below is a reference to a WIP prototype which implements this functionality (prototype)... I have attempted to disrupt as little code as possible and tried to follow the current code structure (for the most part) in the areas I modified. I also studied how Hadoop achieves encrypted shuffle (http://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html) https://github.com/turp1twin/spark/commit/024b559f27945eb63068d1badf7f82e4e7c3621c -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org