This is an automated email from the ASF dual-hosted git repository. dongjoon pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.5 by this push: new b8e2498007a0 [SPARK-47318][CORE][3.5] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices b8e2498007a0 is described below commit b8e2498007a0b4e9a8790ce8a3511427093fa6a7 Author: Dongjoon Hyun <dongj...@apache.org> AuthorDate: Sat Apr 13 16:54:20 2024 -0700 [SPARK-47318][CORE][3.5] Adds HKDF round to AuthEngine key derivation to follow standard KEX practices ### What changes were proposed in this pull request? Backport of SPARK-47318 to v3.5.0 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. Closes #46014 from sweisdb/SPARK-47318-v3.5.0. Lead-authored-by: Dongjoon Hyun <dongj...@apache.org> Co-authored-by: Steve Weis <steve.w...@databricks.com> Signed-off-by: Dongjoon Hyun <dh...@apple.com> --- .../apache/spark/network/crypto/AuthEngine.java | 15 +++++++++- .../java/org/apache/spark/network/crypto/README.md | 17 ++++++++++- .../apache/spark/network/util/TransportConf.java | 9 ++++++ .../spark/network/crypto/AuthEngineSuite.java | 34 ++++++++++++++++++++-- docs/security.md | 12 ++++++++ 5 files changed, 82 insertions(+), 5 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java index 078d9ceb317b..14f0c23fd05f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -41,16 +41,19 @@ import org.apache.spark.network.util.TransportConf; * Exchange, using a pre-shared key to derive an AES-GCM key encrypting key. */ class AuthEngine implements Closeable { + public static final byte[] DERIVED_KEY_INFO = "derivedKey".getBytes(UTF_8); public static final byte[] INPUT_IV_INFO = "inputIv".getBytes(UTF_8); public static final byte[] OUTPUT_IV_INFO = "outputIv".getBytes(UTF_8); private static final String MAC_ALGORITHM = "HMACSHA256"; private static final int AES_GCM_KEY_SIZE_BYTES = 16; private static final byte[] EMPTY_TRANSCRIPT = new byte[0]; + private static final int UNSAFE_SKIP_HKDF_VERSION = 1; private final String appId; private final byte[] preSharedSecret; private final TransportConf conf; private final Properties cryptoConf; + private final boolean unsafeSkipFinalHkdf; private byte[] clientPrivateKey; private TransportCipher sessionCipher; @@ -62,6 +65,9 @@ class AuthEngine implements Closeable { this.preSharedSecret = preSharedSecret.getBytes(UTF_8); this.conf = conf; this.cryptoConf = conf.cryptoConf(); + // This is for backward compatibility with version 1.0 of this protocol, + // which did not perform a final HKDF round. + this.unsafeSkipFinalHkdf = conf.authEngineVersion() == UNSAFE_SKIP_HKDF_VERSION; } @VisibleForTesting @@ -201,6 +207,13 @@ class AuthEngine implements Closeable { byte[] sharedSecret, boolean isClient, byte[] transcript) throws GeneralSecurityException { + byte[] derivedKey = unsafeSkipFinalHkdf ? sharedSecret : // This is for backwards compatibility + Hkdf.computeHkdf( + MAC_ALGORITHM, + sharedSecret, + transcript, + DERIVED_KEY_INFO, + AES_GCM_KEY_SIZE_BYTES); byte[] clientIv = Hkdf.computeHkdf( MAC_ALGORITHM, sharedSecret, @@ -213,7 +226,7 @@ class AuthEngine implements Closeable { 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"); return new TransportCipher( cryptoConf, conf.cipherTransformation(), diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md index 78e7459b9995..5d3584d80462 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md @@ -1,6 +1,9 @@ -Forward Secure Auth Protocol +Forward Secure Auth Protocol v2.0 ============================================== +Summary +------- + This file describes a forward secure authentication protocol which may be used by Spark. This protocol is essentially ephemeral Diffie-Hellman key exchange using Curve25519, referred to as X25519. @@ -77,6 +80,7 @@ Now that the server has the client's ephemeral public key, it can generate its o keypair and compute a shared secret. sharedSecret = X25519.computeSharedSecret(clientPublicKey, serverKeyPair.privateKey()) + derivedKey = HKDF(sharedSecret, salt=transcript, info="deriveKey") With the shared secret, the server will also generate two initialization vectors to be used for inbound and outbound streams. These IVs are not secret and will be bound to the preceding protocol @@ -99,3 +103,14 @@ 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 latest version adds this additional +HKDF to derive `derivedKey`. + +Consequently, Apache Spark instances using v1.0 of this protocol will not negotiate the same key as +instances using v2.0 and will be **unable to send encrypted RPCs** across incompatible versions. For this reason, v1.0 +remains the default to preserve backward-compatibility. \ No newline at end of file diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 45e9994be722..e4966b32fb45 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -212,6 +212,15 @@ public class TransportConf { return conf.getBoolean("spark.network.crypto.enabled", false); } + /** + * Version number to be used by the AuthEngine key agreement protocol. Valid values are 1 or 2. + * The default version is 1 for backward compatibility. Version 2 is recommended for stronger + * security properties. + */ + public int authEngineVersion() { + return conf.getInt("spark.network.crypto.authEngineVersion", 1); + } + /** * The cipher transformation to use for encrypting session data. */ diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java index c6029a70bd61..971e3ef2ff98 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -20,6 +20,7 @@ package org.apache.spark.network.crypto; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; import java.security.GeneralSecurityException; +import java.util.Collections; import java.util.Random; import com.google.crypto.tink.subtle.Hex; @@ -27,6 +28,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.FileRegion; import org.apache.spark.network.util.ByteArrayWritableChannel; +import org.apache.spark.network.util.ConfigProvider; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; import static org.junit.Assert.*; @@ -48,7 +50,10 @@ public class AuthEngineSuite { "fb00000005617070496400000010708451c9dd2792c97c1ca66e6df449ef0000003c64fe899ecdaf458d4" + "e25e9d5c5a380b8e6d1a184692fac065ed84f8592c18e9629f9c636809dca2ffc041f20346eb53db78738" + "08ecad08b46b5ee3ff"; - private static final String sharedKey = + + private static final String derivedKey = "2d6e7a9048c8265c33a8f3747bfcc84c"; + // This key would have been derived for version 1.0 protocol that did not run a final HKDF round. + private static final String unsafeDerivedKey = "31963f15a320d5c90333f7ecf5cf3a31c7eaf151de07fef8494663a9f47cfd31"; private static final String inputIv = "fc6a5dc8b90a9dad8f54f08b51a59ed2"; private static final String outputIv = "a72709baf00785cad6329ce09f631f71"; @@ -56,7 +61,9 @@ public class AuthEngineSuite { @BeforeClass public static void setUp() { - conf = new TransportConf("rpc", MapConfigProvider.EMPTY); + ConfigProvider v2Provider = new MapConfigProvider(Collections.singletonMap( + "spark.network.crypto.authEngineVersion", "2")); + conf = new TransportConf("rpc", v2Provider); } @Test @@ -174,7 +181,28 @@ public class AuthEngineSuite { // Verify that the client will accept an old transcript. client.deriveSessionCipher(clientChallenge, serverResponse); TransportCipher clientCipher = client.sessionCipher(); - assertEquals(Hex.encode(clientCipher.getKey().getEncoded()), sharedKey); + assertEquals(Hex.encode(clientCipher.getKey().getEncoded()), derivedKey); + assertEquals(Hex.encode(clientCipher.getInputIv()), inputIv); + assertEquals(Hex.encode(clientCipher.getOutputIv()), outputIv); + } + } + + @Test + public void testFixedChallengeResponseUnsafeVersion() throws Exception { + ConfigProvider v1Provider = new MapConfigProvider(Collections.singletonMap( + "spark.network.crypto.authEngineVersion", "1")); + TransportConf v1Conf = new TransportConf("rpc", v1Provider); + try (AuthEngine client = new AuthEngine("appId", "secret", v1Conf)) { + byte[] clientPrivateKey = Hex.decode(clientPrivate); + client.setClientPrivateKey(clientPrivateKey); + AuthMessage clientChallenge = + AuthMessage.decodeMessage(ByteBuffer.wrap(Hex.decode(clientChallengeHex))); + AuthMessage serverResponse = + AuthMessage.decodeMessage(ByteBuffer.wrap(Hex.decode(serverResponseHex))); + // Verify that the client will accept an old transcript. + client.deriveSessionCipher(clientChallenge, serverResponse); + TransportCipher clientCipher = client.sessionCipher(); + assertEquals(Hex.encode(clientCipher.getKey().getEncoded()), unsafeDerivedKey); assertEquals(Hex.encode(clientCipher.getInputIv()), inputIv); assertEquals(Hex.encode(clientCipher.getOutputIv()), outputIv); } diff --git a/docs/security.md b/docs/security.md index c5d132f680a4..c0a4b4da0303 100644 --- a/docs/security.md +++ b/docs/security.md @@ -154,6 +154,12 @@ authentication must also be enabled and properly configured. AES encryption uses [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 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. + There is also support for SASL-based encryption, although it should be considered deprecated. It is still required when talking to shuffle services from Spark versions older than 2.2.0. @@ -169,6 +175,12 @@ 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.</td> + <td>3.4.3, 3.5.2</td> +</tr> <tr> <td><code>spark.network.crypto.config.*</code></td> <td>None</td> --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org