Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-19139][core] New auth mechanism for transport library. #16521

Closed
wants to merge 7 commits into from

Conversation

vanzin
Copy link
Contributor

@vanzin vanzin commented Jan 9, 2017

This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.

The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.

The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.

Details of the auth protocol are discussed in the included README.md
file.

This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.

How was this patch tested?

  • Unit tests
  • Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled
  • Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled

This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.

The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.

The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.

Details of the auth protocol are discussed in the included README.md
file.

This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.
@SparkQA
Copy link

SparkQA commented Jan 10, 2017

Test build #71095 has finished for PR 16521 at commit e219c8e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 10, 2017

Hmm, my final cleanup broke some tests, let me fix those...

@SparkQA
Copy link

SparkQA commented Jan 10, 2017

Test build #71104 has finished for PR 16521 at commit a2b3ff6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 10, 2017

/cc @zsxwing @squito @andrewor14 (in case he's still looking at things)

@yhuai
Copy link
Contributor

yhuai commented Jan 10, 2017

@vanzin I have not reviewed this PR yet. Just have two level questions. Is there any change to existing behaviors and settings (compared with Spark 2.1)? Also, does our doc have enough contents to explain how to set those confs and how those work? Thanks!

@vanzin
Copy link
Contributor Author

vanzin commented Jan 10, 2017

Is there any change to existing behaviors and settings (compared with Spark 2.1)?

No. I added some new config names that replace old ones (to have more generic names), but the old names still work.

Also, does our doc have enough contents to explain how to set those confs and how those work?

I think so. I added docs for the new configs, the important old ones were already documented.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71167 has finished for PR 16521 at commit 3894a02.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@squito squito left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not able to sign off on this completely, but this looks great to me. The README.md is really good at explaining what is going on here.

I just have really minor comments.

- ANONCE: the nonce used as the salt when generating the auth key.
- ENC(): an encryption function that uses the cipher and the generated key. This function
will also be used in the definition of other messages below.
- CCHALLENGE: a byte sequence used as a challenge to the server.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo: CHALLENGE


Where:

- CRESPONSE: the server's response to the client challenge.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo: RESPONSE

The default KDF is "PBKDF2WithHmacSHA1". Users should be able to select any algorithm
from those supported by the `javax.crypto.SecretKeyFactory` class, as long as they support
PBEKeySpec when generating keys. The default number of iterations is calculated to take a
resonable amount of time on modern CPUs. See the documentation in TransportConf for more
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

reasonable

enabled. This is supported by the block transfer service and the
RPC endpoints.
Enable encryption using the commons-crypto library for RPC and block transfer service.
Requires <code>spark.authenticate</code> to be enabled.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if spark.authenticate=false, what happens if this is true? It looks like it is just ignored, I think fail-fast would be ideal.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added some for validating that in SparkConf; right now the config keys are scattered all over the code, I'll file a separate bug for cleaning those up.

@SparkQA
Copy link

SparkQA commented Jan 14, 2017

Test build #71347 has finished for PR 16521 at commit ee9d232.

  • This patch fails from timeout after a configured wait of `250m`.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 14, 2017

retest this please

@SparkQA
Copy link

SparkQA commented Jan 14, 2017

Test build #71355 has finished for PR 16521 at commit ee9d232.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 14, 2017

Test build #71356 has finished for PR 16521 at commit ee9d232.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 17, 2017

Will leave this open for a couple more days, but would appreciate more eyes.

Copy link

@jsoltren jsoltren left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems reasonable to me.

- Encrypting AES session keys with 3DES doesn't solve the issue, since the weakest link
in the negotiation would still be MD5 and 3DES.

The protocol assumes that the shared secret is generated and distributed in a secure manner.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It might be helpful to mention the current shared secret generation and distribution mechanisms to drive the point that these are, hopefully, stronger than DIGEST-MD5 or possibly even AES.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is discussed in SecurityManager.scala.

@zsxwing
Copy link
Member

zsxwing commented Jan 20, 2017

Sorry for the delay. Looking at it now.

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Need to leave now. Left some nits. The core changes look good to me. Nice PR. Will finish my review at the weekend.


val encryptionEnabled = get(NETWORK_ENCRYPTION_ENABLED) || get(SASL_ENCRYPTION_ENABLED)
require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED),
s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's unclear in the doc that what will be used when both spark.network.crypto.enabled and spark.authenticate.enableSaslEncryption are true. It's better to just disable this case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Disable what case?

You need to be able to configure them separately, and if for some weird reason you want RPC encryption but don't want shuffle encryption when talking to an old shuffle service, these settings allow that.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense.

// OK to switch back to SASL (because the server doesn't speak the new protocol). So
// try it anyway, and in the worst case things will fail again.
if (conf.saslFallback()) {
LOG.debug("New auth protocol failed, trying SASL.", e);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: sometimes, it's just because the server config is wrong and a warning is better to help the user find out it.

LOG.debug("Received new auth challenge for client {}.", channel.remoteAddress());
} catch (RuntimeException e) {
if (conf.saslFallback()) {
LOG.debug("Failed to parse new auth challenge, reverting to SASL for client {}.",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: debug -> warn

ByteBuf buf = Unpooled.wrappedBuffer(buffer);

if (buf.readByte() != TAG_BYTE) {
throw new IllegalArgumentException("Expected ServerChallenge, received something else.");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: ServerChallenge -> ServerResponse


public final String appId;
public final String kdf;
public int iterations;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: miss final

malicious "proxy" between endpoints, the attacker won't be able to read any of the data exchanged
between client and server, nor insert arbitrary commands for the server to execute.

* Replay attacks: the use of nonces when generating keys prevents an attacker from being able to
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The server doesn't verify a nonce was used or not, so it don't prevents replay attacks. Right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is explained in the paragraph after the bullet list. The server always generates new nonces for sessions, so replaying the challenge will not allow an attacker to establish a session.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah. I didn't read the codes correctly.

public int maxSaslEncryptedBlockSize() {
return Ints.checkedCast(JavaUtils.byteStringAsBytes(
conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k")));
public boolean aesEncryptionEnabled() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: rename this method to a general name?


TransportClient client;
TransportServer server;
Channel serverChannel;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: volatile

TransportClient client;
TransportServer server;
Channel serverChannel;
AuthRpcHandler authRpcHandler;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: volatile


ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000);
assertEquals("Pong", JavaUtils.bytesToString(reply));
assertTrue(ctx.authRpcHandler.doDelegate);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: please also check delegate type to ensure it doesn't use sasl

@zsxwing
Copy link
Member

zsxwing commented Jan 23, 2017

Made one pass. Looks good overall. Just some nits.

@SparkQA
Copy link

SparkQA commented Jan 23, 2017

Test build #71869 has finished for PR 16521 at commit 718247e.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 23, 2017

Weird error, the code is there... retest this please

@SparkQA
Copy link

SparkQA commented Jan 23, 2017

Test build #71870 has finished for PR 16521 at commit 718247e.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor Author

vanzin commented Jan 23, 2017

need to rebase to current master...

@zsxwing
Copy link
Member

zsxwing commented Jan 23, 2017

LGTM pending tests

@SparkQA
Copy link

SparkQA commented Jan 23, 2017

Test build #71871 has finished for PR 16521 at commit 39df4b3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Jan 24, 2017

Thanks. Merging to master.

@asfgit asfgit closed this in 8f3f73a Jan 24, 2017
@vanzin vanzin deleted the SPARK-19139 branch January 27, 2017 01:16
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.

The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.

The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.

Details of the auth protocol are discussed in the included README.md
file.

This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.

## How was this patch tested?

- Unit tests
- Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled
- Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes apache#16521 from vanzin/SPARK-19139.
cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 15, 2017
This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.

The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.

The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.

Details of the auth protocol are discussed in the included README.md
file.

This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.

## How was this patch tested?

- Unit tests
- Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled
- Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes apache#16521 from vanzin/SPARK-19139.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants