You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "sweisdb (via GitHub)" <gi...@apache.org> on 2024/03/07 19:03:30 UTC

[PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation [spark]

sweisdb opened a new pull request, #45425:
URL: https://github.com/apache/spark/pull/45425

   ### What changes were proposed in this pull request?
   
   This change adds an additional pass through a key derivation function (KDF) to the key exchange protocol in `AuthEngine`. Currently, it uses the shared secret from a bespoke key negotiation protocol directly. This is an encoded X coordinate on the X25519 curve. It is atypical and not recommended to use that coordinate directly as a key, but rather to pass it to an KDF.
   
   Note, Spark now supports TLS for RPC calls. It is preferable to use that rather than the bespoke AES RPC encryption implemented by `AuthEngine` and `TransportCipher`.
   
   ### Why are the changes needed?
   
   This follows best practices of key negotiation protocols. The encoded X coordinate is not guaranteed to be uniformly distributed over the 32-byte key space. Rather, we pass it through a HKDF function to map it uniformly to a 16-byte key space.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   Exiting tests under:
   `build/sbt "network-common/test:testOnly"`
   
   Specifically:
   `build/sbt "network-common/test:testOnly org.apache.spark.network.crypto.AuthEngineSuite"`
   `build/sbt "network-common/test:testOnly org.apache.spark.network.crypto.AuthIntegrationSuite"`
   
   ### Was this patch authored or co-authored using generative AI tooling?
   
   No.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525398491


##########
docs/security.md:
##########
@@ -149,24 +149,32 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and should be considered
+deprecated in favor of SSL.
 
 If both options are enabled in the configuration, the SSL based RPC encryption takes precedence
 and the AES based encryption will not be used (and a warning message will be emitted).
 
-## AES based Encryption
+## SSL Encryption (Preferred)
+
+Spark supports SSL based encryption for RPC connections. Please refer to the SSL Configuration
+section below to understand how to configure it. The SSL settings are mostly similar across the UI
+and RPC, however there are a few additional settings which are specific to the RPC implementation.
+The RPC implementation uses Netty under the hood (while the UI uses Jetty), which supports a
+different set of options.
+
+Unlike the other SSL settings for the UI, the RPC SSL is *not* automatically enabled if

Review Comment:
   Since this layer is completely irrelevant to UI, we don't need to say like this. Shall we omit this paragraph?
   > Unlike the other SSL settings for the UI



##########
docs/security.md:
##########
@@ -149,24 +149,32 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and should be considered
+deprecated in favor of SSL.
 
 If both options are enabled in the configuration, the SSL based RPC encryption takes precedence
 and the AES based encryption will not be used (and a warning message will be emitted).
 
-## AES based Encryption
+## SSL Encryption (Preferred)
+
+Spark supports SSL based encryption for RPC connections. Please refer to the SSL Configuration
+section below to understand how to configure it. The SSL settings are mostly similar across the UI
+and RPC, however there are a few additional settings which are specific to the RPC implementation.
+The RPC implementation uses Netty under the hood (while the UI uses Jetty), which supports a
+different set of options.
+
+Unlike the other SSL settings for the UI, the RPC SSL is *not* automatically enabled if

Review Comment:
   Since this layer is completely irrelevant to UI, we don't need to say like this. Shall we omit this paragraph?
   > Unlike the other SSL settings for the UI



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1556042923


##########
docs/security.md:
##########
@@ -149,30 +149,44 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   (Edit: I misread your comment as asking to remove text, not questioning why I removed text.) 
   
   I think the original text is rationalizing why there is an ad hoc protocol here instead in the first place. We should make it clear that TLS is strongly preferred and make it easy to migrate.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1554526813


##########
docs/security.md:
##########
@@ -149,30 +149,44 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   Looking at the earlier text, we are dropping some verbiage from it - any particular reason for it ?
   Specifically `"However, the AES based encryption is simpler to configure and may be preferred if the only requirement is that data be encrypted in transit."` for ex.
   
   We have made our recommendation clear, and rest should be left to users to determine - and make the right tradeoff.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552128652


##########
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java:
##########
@@ -213,6 +213,11 @@ public boolean encryptionEnabled() {
     return conf.getBoolean("spark.network.crypto.enabled", false);
   }
 
+  /**
+   * Version number to be used by the AuthEngine key agreement protocol. Value values are 1 or 2.

Review Comment:
   Also, shall we mentioned that the default is 1 intentionally?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050107925

   Looks like the CI build is passing: https://github.com/sweisdb/spark/actions/runs/8623329437


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050143917

   Ya, it looks like that, @sweisdb .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2030688115

   I had a bad push that pulled in a lot of other unrelated commits. Will fix.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525393098


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +114,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` and was
+directly using the encoded X coordinate as key material. This is atypical and standard practice is to pass that shared
+coordinate through an HKDF. The current version, v1.1, adds this additional HKDF to
+derive `derivedKey`.

Review Comment:
   +1 for @mridulm 's recommendation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552337398


##########
common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java:
##########
@@ -48,15 +50,19 @@ public class AuthEngineSuite {
       "fb00000005617070496400000010708451c9dd2792c97c1ca66e6df449ef0000003c64fe899ecdaf458d4" +
       "e25e9d5c5a380b8e6d1a184692fac065ed84f8592c18e9629f9c636809dca2ffc041f20346eb53db78738" +
       "08ecad08b46b5ee3ff";
-  private static final String sharedKey =
-      "31963f15a320d5c90333f7ecf5cf3a31c7eaf151de07fef8494663a9f47cfd31";
+  private static final String derivedKey = "2d6e7a9048c8265c33a8f3747bfcc84c";
+  // This key would have been derived from an older version of the protocol that did not run a final HKDF pass
+  private static final String unsafeDerivedKey = "31963f15a320d5c90333f7ecf5cf3a31c7eaf151de07fef8494663a9f47cfd31";
+
   private static final String inputIv = "fc6a5dc8b90a9dad8f54f08b51a59ed2";
   private static final String outputIv = "a72709baf00785cad6329ce09f631f71";
   private static TransportConf conf;
 
   @BeforeAll
   public static void setUp() {
-    conf = new TransportConf("rpc", MapConfigProvider.EMPTY);
+    ConfigProvider v2Provider = new MapConfigProvider(Collections.singletonMap(
+            "spark.network.crypto.authEngineVersion", "2"));

Review Comment:
   We test both v1 and v2. See below where I specifically give it a config with "1".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525394252


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +114,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` and was
+directly using the encoded X coordinate as key material. This is atypical and standard practice is to pass that shared
+coordinate through an HKDF. The current version, v1.1, adds this additional HKDF to
+derive `derivedKey`.
+
+Consequently, older Spark versions using v1.0 of this protocol will not negotiate the same key as
+Spark versions using v1.1 and will be **unable to send encrypted RPCs** across incompatible versions.

Review Comment:
   I assume that Apache Spark community follows `Semantic Versioning` by default. If this is incompatible, shall we use `v2.0` instead of `v1.1`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-1997757960

   @dongjoon-hyun Will do. I would clarify that the best practice here would be to use TLS and not use a bespoke key exchange protocol. 
   
   But if we are going to use a bespoke key exchange protocol, we need to run the output of the key exchange through a KDF.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1548843062


##########
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java:
##########
@@ -213,6 +213,11 @@ public boolean encryptionEnabled() {
     return conf.getBoolean("spark.network.crypto.enabled", false);
   }
 
+  /**
+   * Version number to be used by the AuthEngine key agreement protocol. Value values are 1 or 2.
+   */
+  public int authEngineVersion() { return conf.getInt("spark.network.crypto.authEngineVersion", 2); }

Review Comment:
   +1 for the above comment, too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-1997934816

   That sounds fine re doc update @sweisdb 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1548842531


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -224,7 +236,7 @@ private TransportCipher generateTransportCipher(
 
   private byte[] getTranscript(AuthMessage... encryptedPublicKeys) {
     ByteBuf transcript = Unpooled.buffer(
-        Arrays.stream(encryptedPublicKeys).mapToInt(k -> k.encodedLength()).sum());
+        Arrays.stream(encryptedPublicKeys).mapToInt(AuthMessage::encodedLength).sum());

Review Comment:
   +1 for @mridulm 's comment. This should be reverted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2040971830

   Can you please fix the linter related build errors @sweisdb ? Thx


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552041848


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -224,7 +236,7 @@ private TransportCipher generateTransportCipher(
 
   private byte[] getTranscript(AuthMessage... encryptedPublicKeys) {
     ByteBuf transcript = Unpooled.buffer(
-        Arrays.stream(encryptedPublicKeys).mapToInt(k -> k.encodedLength()).sum());
+        Arrays.stream(encryptedPublicKeys).mapToInt(AuthMessage::encodedLength).sum());

Review Comment:
   Reverted. Good call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552132141


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +103,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` to derive
+a key (i.e. `derivedKey`) and was directly using the encoded X coordinate as key material. This is atypical and
+standard practice is to pass that shared coordinate through an HKDF. The current version, v2.0, adds this additional
+HKDF to derive `derivedKey`.
+
+Consequently, older Spark versions using v1.0 of this protocol will not negotiate the same key as
+Spark versions using v2.0 and will be **unable to send encrypted RPCs** across incompatible versions.

Review Comment:
   Shall we add a sentence to describe Apache Spark sticks to `v1.0` by default due to the above reason?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050393781

   Merged to master.
   Thanks for fixing this @sweisdb !
   Thanks for reviewing @dongjoon-hyun :-)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525405081


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.

Review Comment:
   To @sweisdb , may I ask when you think we can delete this? If you think `Spark 5`, please remove this claim. It's too far. In Apache Spark community, deprecation doesn't mean the removal.
   > and removed in the long-term.
   
   FYI, here is the guideline.
   - https://spark.apache.org/versioning-policy.html
   https://spark.apache.org/versioning-policy.html



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1526548877


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.
+
+See
+the [Spark security documentation](https://github.com/apache/spark/blob/master/docs/security.md#ssl-encryption) for
+more information on how to configure TLS.

Review Comment:
   Given that SASL support still exists for backward compatibility, I am skeptical that this will actually be removed. I'll delete that comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1526551200


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1

Review Comment:
   Will fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1546923268


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -213,7 +220,7 @@ private TransportCipher generateTransportCipher(
         transcript,  // Passing this as the HKDF salt
         OUTPUT_IV_INFO,  // This is the HKDF info field used to differentiate IV values
         AES_GCM_KEY_SIZE_BYTES);
-    SecretKeySpec sessionKey = new SecretKeySpec(sharedSecret, "AES");
+    SecretKeySpec sessionKey = new SecretKeySpec(derivedKey, "AES");

Review Comment:
   I added a flag to let someone override this and revert back to the original protocol.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2040972278

   Btw, we would not be able to directly merge this PR into older versions - given the reference to SSL in the docs.
   So unfortunately, we will need a follow up PR for 3.5.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1526551964


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.

Review Comment:
   After seeing that we're still supporting legacy SASL, I agree that this will likely not be removed any time in the forseeable future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1546872088


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.

Review Comment:
   Fixing this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-1997843099

   I've updated the overall Security doc to indicate that TLS is preferred to using the legacy RPC encryption. I also updated the AuthEngine README to talk about incompatibility.
   
   I retroactively made up v1.0 and v1.1 of the key exchange protocol to talk about this. It would probably be more understandable to talk about Spark versions. Should we talk about 4.0 in the context of this change and amend the docs if and when it's backported?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1548815203


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +103,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` and was

Review Comment:
   nit:
   
   ```suggestion
   The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` (to generate the `derivedKey`) and was
   ```



##########
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java:
##########
@@ -213,6 +213,11 @@ public boolean encryptionEnabled() {
     return conf.getBoolean("spark.network.crypto.enabled", false);
   }
 
+  /**
+   * Version number to be used by the AuthEngine key agreement protocol. Value values are 1 or 2.
+   */
+  public int authEngineVersion() { return conf.getInt("spark.network.crypto.authEngineVersion", 2); }

Review Comment:
   Make the default 1, to preserve the existing behavior, and allowing backport to patch versions.
   We can document to set this explicitly to to `2` to mitigate this issue.
   
   In a later minor version, we can flip this to `2` by default.



##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -224,7 +236,7 @@ private TransportCipher generateTransportCipher(
 
   private byte[] getTranscript(AuthMessage... encryptedPublicKeys) {
     ByteBuf transcript = Unpooled.buffer(
-        Arrays.stream(encryptedPublicKeys).mapToInt(k -> k.encodedLength()).sum());
+        Arrays.stream(encryptedPublicKeys).mapToInt(AuthMessage::encodedLength).sum());

Review Comment:
   Let us minimize optional changes in this PR (this by itself is fine - but in general, we should reduce the scope of change), given we want to backport to older versions and want to minimize risk.



##########
docs/security.md:
##########
@@ -149,24 +149,32 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   ```suggestion
   RPC authentication to also be enabled. This method uses a bespoke protocol with an older v1 and an updated v2 version. <add details about v1 vs v2>
   ```
   
   Also, update the config section below on how to configure `spark.network.crypto.authEngineVersion`, and a ref to that here: along with incompatibility between `1` and `2` (and all versions prior being `1` retroactively - without a way to configure it before `3.5.2` and `3.4.3`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552339596


##########
docs/security.md:
##########
@@ -188,6 +202,11 @@ The following table describes the different options available for configuring th
   </td>
   <td>2.2.0</td>
 </tr>
+<tr>
+  <td><code>spark.network.crypto.authEngineVersion</code></td>
+  <td>1</td>
+  <td>Version of AES-based RPC encryption to use. Valid versions are 1 or 2. Version 2 is recommended. Added in 4.0.0</td>

Review Comment:
   Good catch. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050146218

   BTW, @sweisdb , could you make two backporting PR to `branch-3.5` and `branch-3.4`?
   
   I'm the release manager for Apache Spark 3.4.3 and targeting RC1 next Monday. I want to deliver this patch via Apache Spark 3.4.3.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525231740


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.
+
+See
+the [Spark security documentation](https://github.com/apache/spark/blob/master/docs/security.md#ssl-encryption) for
+more information on how to configure TLS.

Review Comment:
   I am not sure if we should announce deprecation given TLS is going to come out only in 4.0 ... we can recommend users to move to TLS.
   
   Would like feedback from others on this though ... @dongjoon-hyun, @srowen, @JoshRosen 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525236753


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -213,7 +220,7 @@ private TransportCipher generateTransportCipher(
         transcript,  // Passing this as the HKDF salt
         OUTPUT_IV_INFO,  // This is the HKDF info field used to differentiate IV values
         AES_GCM_KEY_SIZE_BYTES);
-    SecretKeySpec sessionKey = new SecretKeySpec(sharedSecret, "AES");
+    SecretKeySpec sessionKey = new SecretKeySpec(derivedKey, "AES");

Review Comment:
   Let us flag guard this change - given we want to backport to 3.x patch releases as well as giving users migration opportunities. This flag can be flipped to default in a subsequent 4.x minor version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525394625


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1

Review Comment:
   `v2.0`? (https://github.com/apache/spark/pull/45425/files/4204dd1f03cb86f4f6e082d2c4b8bd5f402ad12f..7ace803c40ab50c03e4af1c3733499668fb340d5#r1525394252)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552128114


##########
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java:
##########
@@ -213,6 +213,11 @@ public boolean encryptionEnabled() {
     return conf.getBoolean("spark.network.crypto.enabled", false);
   }
 
+  /**
+   * Version number to be used by the AuthEngine key agreement protocol. Value values are 1 or 2.

Review Comment:
   `Value values`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552336712


##########
common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java:
##########
@@ -213,6 +213,11 @@ public boolean encryptionEnabled() {
     return conf.getBoolean("spark.network.crypto.enabled", false);
   }
 
+  /**
+   * Version number to be used by the AuthEngine key agreement protocol. Value values are 1 or 2.

Review Comment:
   Good catch. Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050275736

   Thank you so much!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552133989


##########
common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java:
##########
@@ -48,15 +50,19 @@ public class AuthEngineSuite {
       "fb00000005617070496400000010708451c9dd2792c97c1ca66e6df449ef0000003c64fe899ecdaf458d4" +
       "e25e9d5c5a380b8e6d1a184692fac065ed84f8592c18e9629f9c636809dca2ffc041f20346eb53db78738" +
       "08ecad08b46b5ee3ff";
-  private static final String sharedKey =
-      "31963f15a320d5c90333f7ecf5cf3a31c7eaf151de07fef8494663a9f47cfd31";
+  private static final String derivedKey = "2d6e7a9048c8265c33a8f3747bfcc84c";
+  // This key would have been derived from an older version of the protocol that did not run a final HKDF pass
+  private static final String unsafeDerivedKey = "31963f15a320d5c90333f7ecf5cf3a31c7eaf151de07fef8494663a9f47cfd31";
+
   private static final String inputIv = "fc6a5dc8b90a9dad8f54f08b51a59ed2";
   private static final String outputIv = "a72709baf00785cad6329ce09f631f71";
   private static TransportConf conf;
 
   @BeforeAll
   public static void setUp() {
-    conf = new TransportConf("rpc", MapConfigProvider.EMPTY);
+    ConfigProvider v2Provider = new MapConfigProvider(Collections.singletonMap(
+            "spark.network.crypto.authEngineVersion", "2"));

Review Comment:
   I'm wondering if this means that we don't test `v1` from now.
   
   We had better keep the test coverage for v1 in addition to v2.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552134883


##########
docs/security.md:
##########
@@ -149,30 +149,44 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.
 
 If both options are enabled in the configuration, the SSL based RPC encryption takes precedence
 and the AES based encryption will not be used (and a warning message will be emitted).
 
-## AES based Encryption
+## SSL Encryption (Preferred)
+
+Spark supports SSL based encryption for RPC connections. Please refer to the SSL Configuration
+section below to understand how to configure it. The SSL settings are mostly similar across the UI
+and RPC, however there are a few additional settings which are specific to the RPC implementation.
+The RPC implementation uses Netty under the hood (while the UI uses Jetty), which supports a
+different set of options.
+
+Unlike the other SSL settings for the UI, the RPC SSL is *not* automatically enabled if
+`spark.ssl.enabled` is set. It must be explicitly enabled, to ensure a safe migration path for users
+upgrading Spark versions.
+
+## AES-based Encryption (Legacy)
 
 Spark supports AES-based encryption for RPC connections. For encryption to be enabled, RPC
 authentication must also be enabled and properly configured. AES encryption uses the
 [Apache Commons Crypto](https://commons.apache.org/proper/commons-crypto/) library, and Spark's
 configuration system allows access to that library's configuration for advanced users.
 
+This legacy protocol has two mutually incompatible versions. Version 1 omits applying key derivation function
+(KDF) to the key exchange protocol's output, while version 2 applies a KDF to ensure that the derived session
+key is uniformly distributed. Version 1 is default for backward compatibility. It is **recommended to use version 2**
+for better security properties. The version can be configured by setting `spark.network.crypto.authEngineVersion` to
+1 or 2 respectively.

Review Comment:
   Thank you for the above description.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552131042


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +103,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` to derive
+a key (i.e. `derivedKey`) and was directly using the encoded X coordinate as key material. This is atypical and
+standard practice is to pass that shared coordinate through an HKDF. The current version, v2.0, adds this additional

Review Comment:
   `The current version, v2.0,` could mislead the users. Let's say `The latest version` instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552137285


##########
docs/security.md:
##########
@@ -188,6 +202,11 @@ The following table describes the different options available for configuring th
   </td>
   <td>2.2.0</td>
 </tr>
+<tr>
+  <td><code>spark.network.crypto.authEngineVersion</code></td>
+  <td>1</td>
+  <td>Version of AES-based RPC encryption to use. Valid versions are 1 or 2. Version 2 is recommended. Added in 4.0.0</td>

Review Comment:
   `Added in 4.0.0` looks like a typo. This will break this HTML table. In other words, we need to have two lines of `<td></td>`. One for description and the other for version. Please remove ` Added in 4.0.0` from here and create a new line, `<td>4.0.0</td>`, in order to follow the other configuration's style.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1552042143


##########
docs/security.md:
##########
@@ -149,24 +149,32 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   Updated in the section about the Legacy protocol.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2046170137

   > Ya, please use `dev/lint-java` script and fix the following lines, @sweisdb .
   > 
   > ```
   > $ dev/lint-java
   > Using `mvn` from path: /opt/homebrew/bin/mvn
   > Using SPARK_LOCAL_IP=localhost
   > Checkstyle checks failed at following occurrences:
   > [ERROR] src/main/java/org/apache/spark/network/crypto/AuthEngine.java:[68] (sizes) LineLength: Line is longer than 100 characters (found 113).
   > [ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[218] (sizes) LineLength: Line is longer than 100 characters (found 116).
   > [ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[220] (sizes) LineLength: Line is longer than 100 characters (found 101).
   > [ERROR] src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java:[54] (sizes) LineLength: Line is longer than 100 characters (found 110).
   > [ERROR] src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java:[55] (sizes) LineLength: Line is longer than 100 characters (found 116).
   > ```
   
   Addressed the lint issues.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1525236753


##########
common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java:
##########
@@ -213,7 +220,7 @@ private TransportCipher generateTransportCipher(
         transcript,  // Passing this as the HKDF salt
         OUTPUT_IV_INFO,  // This is the HKDF info field used to differentiate IV values
         AES_GCM_KEY_SIZE_BYTES);
-    SecretKeySpec sessionKey = new SecretKeySpec(sharedSecret, "AES");
+    SecretKeySpec sessionKey = new SecretKeySpec(derivedKey, "AES");

Review Comment:
   Let us flag guard this change - given we want to backport to 3.x patch releases as well as giving users migration opportunities. This flag can be flipped to default in a subsequent minor version.



##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -1,6 +1,20 @@
-Forward Secure Auth Protocol
+Forward Secure Auth Protocol v1.1
 ==============================================
 
+Deprecation Notice
+------------------
+This is a bespoke key exchange protocol that was implemented before Spark supported TLS (aka SSL) for RPC
+calls. It is recommended that Spark users upgrade to using TLS for RPC calls between Spark processes. This protocol
+will be deprecated and removed in the long-term.
+
+See
+the [Spark security documentation](https://github.com/apache/spark/blob/master/docs/security.md#ssl-encryption) for
+more information on how to configure TLS.

Review Comment:
   I am not sure if we should announce deprecation given TLS is going to come out only in 4.0 ... we can recommend users to move to TLS though.
   
   Would like feedback from others on this though ... @dongjoon-hyun, @srowen, @JoshRosen 



##########
common/network-common/src/main/java/org/apache/spark/network/crypto/README.md:
##########
@@ -99,3 +114,13 @@ sessions. It would, however, allow impersonation of future sessions.
 In the event of a pre-shared key compromise, messages would still be confidential from a passive
 observer. Only active adversaries spoofing a session would be able to recover plaintext.
 
+Security Changes & Compatibility
+-------------
+
+The original version of this protocol, retroactively called v1.0, did not apply an HKDF to `sharedSecret` and was
+directly using the encoded X coordinate as key material. This is atypical and standard practice is to pass that shared
+coordinate through an HKDF. The current version, v1.1, adds this additional HKDF to
+derive `derivedKey`.

Review Comment:
   Essentially, this is backwardly incompatible with 3.x  on the wire - making adoption of 4.x in secure env all-or-nothing
   Given this, I am leaning towards flag guarding the change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1556106453


##########
docs/security.md:
##########
@@ -149,30 +149,44 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   I put the original text back in. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on code in PR #45425:
URL: https://github.com/apache/spark/pull/45425#discussion_r1556042923


##########
docs/security.md:
##########
@@ -149,30 +149,44 @@ secret file agrees with the executors' secret file.
 
 # Network Encryption
 
-Spark supports two mutually exclusive forms of encryption for RPC connections.
+Spark supports two mutually exclusive forms of encryption for RPC connections:
 
-The first is an AES-based encryption which relies on a shared secret, and thus requires
-RPC authentication to also be enabled.
+The **preferred method** uses TLS (aka SSL) encryption via Netty's support for SSL. Enabling SSL
+requires keys and certificates to be properly configured. SSL is standardized and considered more
+secure.
 
-The second is an SSL based encryption mechanism utilizing Netty's support for SSL. This requires
-keys and certificates to be properly configured. It can be used with or without the authentication
-mechanism discussed earlier.
-
-One may prefer to use the SSL based encryption in scenarios where compliance mandates the usage
-of specific protocols; or to leverage the security of a more standard encryption library. However,
-the AES based encryption is simpler to configure and may be preferred if the only requirement
-is that data be encrypted in transit.
+The legacy method is an AES-based encryption mechanism relying on a shared secret. This requires
+RPC authentication to also be enabled. This method uses a bespoke protocol and it is recommended
+to use SSL instead.

Review Comment:
   I will take this out. I think it is rationalizing why there is an ad hoc protocol here instead in the first place. I agree that we should make it clear that TLS is strongly preferred and make it easy to migrate.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][Security] Adds HKDF round to AuthEngine key derivation [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-1996748945

   +CC @vanzin, @srowen 
   
   Can you also include the documentation change in the PR to call out the incompatibility introduced due to this ? Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2000725355

   Got it, @mridulm .
   > For context to reviewers, the plan is to backport this to 3.x as well given security implications - that is, we will will be making an incompatible wire protocol change.
   
   If this is for Apache Spark 3.5.2 and 3.4.3, cc @cloud-fan , @HyukjinKwon , too


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050144383

   Hi, @mridulm . Could you switch your request change?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm closed pull request #45425: [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices
URL: https://github.com/apache/spark/pull/45425


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47318][CORE] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices [spark]

Posted by "sweisdb (via GitHub)" <gi...@apache.org>.
sweisdb commented on PR #45425:
URL: https://github.com/apache/spark/pull/45425#issuecomment-2050274536

   @dongjoon-hyun:
   v3.5.0 backport here: https://github.com/apache/spark/pull/46014
   v3.4.0 backport here: https://github.com/apache/spark/pull/46015


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org