Skip to content

Conversation

@chenjunjiedada
Copy link
Contributor

What changes were proposed in this pull request?

DIGEST-MD5 mechanism is used for SASL authentication and secure communication. DIGEST-MD5 mechanism supports 3DES, DES, and RC4 ciphers. However, 3DES, DES and RC4 are slow relatively.

AES provide better performance and security by design and is a replacement for 3DES according to NIST. Apache Common Crypto is a cryptographic library optimized with AES-NI, this patch employ Apache Common Crypto as enc/dec backend for SASL authentication and secure channel to improve spark RPC.

How was this patch tested?

Unit tests and Integration test.

DIGEST-MD5 mechanism is used for SASL authentication and secure communication. DIGEST-MD5 mechanism supports 3DES, DES, and RC4 ciphers. However, 3DES, DES and RC4 are slow relatively.

AES provide better performance and security by design and is a replacement for 3DES according to NIST. Apache Common Crypto is a cryptographic library optimized with AES-NI, this patch employ Apache Common Crypto as enc/dec backend for SASL authentication and secure channel to improve spark RPC.
<artifactId>slf4j-log4j12</artifactId>
<scope>test</scope>
</dependency>
<dependency>

Choose a reason for hiding this comment

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

Can we move this dependency to L62 since the dependency around is test only. You can see the comment in L64 and L80

* authentication.
*/
public class SaslServerBootstrap implements TransportServerBootstrap {

Choose a reason for hiding this comment

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

No need for the changes in this file.

@rxin
Copy link
Contributor

rxin commented Sep 21, 2016

Can you change the title to say "[SPARK-13331] AES support for over-the-wire encryption"?

@chenjunjiedada chenjunjiedada changed the title [Spark-13331][network] Spark RPC encryption optimization [SPARK-13331] AES support for over-the-wire encryption Sep 21, 2016
Remove some unnecessary import and changes.
@vanzin
Copy link
Contributor

vanzin commented Sep 22, 2016

ok to test

@SparkQA
Copy link

SparkQA commented Sep 22, 2016

Test build #65779 has finished for PR 15172 at commit da8aee6.

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

@vanzin
Copy link
Contributor

vanzin commented Sep 22, 2016

That's a weird error... complaining about classes that really should exist. retest this please

@SparkQA
Copy link

SparkQA commented Sep 22, 2016

Test build #65791 has finished for PR 15172 at commit da8aee6.

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

@vanzin
Copy link
Contributor

vanzin commented Sep 22, 2016

Replying to myself:
http://stackoverflow.com/questions/14935447/why-am-i-getting-package-javax-crypto-does-not-exist

Seems like we need a small tweak in the build code. I'll send a separate PR for that.

@vanzin
Copy link
Contributor

vanzin commented Sep 23, 2016

retest this please

@SparkQA
Copy link

SparkQA commented Sep 23, 2016

Test build #65813 has finished for PR 15172 at commit da8aee6.

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

Copy link
Contributor

@vanzin vanzin left a comment

Choose a reason for hiding this comment

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

My main concern here is that I was expecting a lot of this code to not be necessary. I was hoping commons-crypto would be providing easier APIs to do a lot of this stuff.

Another is that I think the internal API here should be using ByteBuffer to avoid unnecessary copies in the AES path.

<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-crypto</artifactId>
<version>${commons-crypto.version}</version>
Copy link
Contributor

Choose a reason for hiding this comment

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

version is already in the root pom, no need to duplicate it.

* @return the new encrypted byte array.
* @throws SaslException if error happens
*/
public byte[] wrap(byte[] data, int offset, int len) throws SaslException {
Copy link
Contributor

@vanzin vanzin Sep 23, 2016

Choose a reason for hiding this comment

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

I think it would be better to modify SaslEncryption to work with ByteBuffer (or event ByteBuf) instead of byte[]; that way the AES implementation would be more efficient. This should be pretty easy for the decryption side, maybe a little more complicated on the encrypting side.

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 suppose you were talking about SaslEncrytionbackend. Actually, the real underlying interfaces of wrap/unwrap are defined by SaslClient which provided by java security.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

wrap/unwrap APIs are defined by SaslClient interface of Java Security.

Copy link
Contributor

Choose a reason for hiding this comment

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

See previous comment. You do not need to implement JDK interfaces here, especially because they don't really add any needed functionality.

They're needed for the SASL encryption backend because that's how you do it with those APIs. It doesn't need the AES encryption code needs to implement those APIs too.

byte[] msg = new byte[decrypted.length - 10];
byte[] mac = new byte[10];

System.arraycopy(decrypted, msg.length, mac, 0, 10);
Copy link
Contributor

@vanzin vanzin Sep 23, 2016

Choose a reason for hiding this comment

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

There's a lot of allocation / copying going on here... using ByteBuffer, or even sticking to byte[] but with APIs that allow you to provide offset + count, would be better.

/**
* AES cipher for encryption and decryption.
*/
public class SparkAesCipher {
Copy link
Contributor

Choose a reason for hiding this comment

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

There's a lot of code in this class that I would have expected to be in commons-crypto; after all, the intent of that library is to make it easy to use crypto, and the code in this class definitely does not look "easy".

Why isn't the library handling most of this? Can we make enhancements to the library first so that it better covers this use 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.

I suppose the "extra" code you mentioned is about the integrity helper class and some padding operation. The padding operation should be removed since it is not a correct use case., I will remove it accordingly. While the Integrity helper class is more related to SASL compared to common crypto library to some extend, the sequence number and HMAC are more like SASL transport context. I think common crypto library, as a general cipher might not appropriate to implement it. I think it would be better to retain it and do some optimization. What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, mostly the HMAC and sequence number. On second though, they seem completely unnecessary. Those are better left to other layers in the networking stack.

Note the goal here is not to implement "SASL encryption" using AES; it's just to provide AES encryption. You do not need to implement it using the JDK's SASL interfaces; once authentication is done, you can just use the most optimized API for the encryption library you're using. In the case of AES/commons crypto, that should be a simple netty channel handler, not an implementation that implements the JDK's wrap / unwrap for no reason.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Seq number and HMAC are necessary parts of SASL encryption, Seq number can make same data with different offsets different, and HMAC is used for integrity check, see com.sun.security.sasl.digest.DigestMD5Base class from JDK as example.

JDK's SASL framework doesn't provide interface to transfer AES cipher, it creates cipher interanlly. For example DigestMD5 client create cipher internally (DigestMD5Base.java:1219). However, it provides SaslEncryptBackend interface for user to customize the SASL client/server, that's why the patch implements wrap/unwrap.

Another way to make patch clean and clear is to add AES cipher support in JDK's Digest-MD5 mechanism or change JDK SASL framework to provide a interface support registering customized cipher, but that would be slow and depends on JDK release.

Copy link
Contributor

Choose a reason for hiding this comment

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

You're missing my point.

The goal is not to implement "SASL encryption". The goal is to implement encryption.

The AES encryption being added does NOT need to follow any of the JVM's SASL APIs, because that's not the goal.

Basically, when using AES, you're using SASL strictly for authentication.

  • open connection
  • authenticate using SASL
  • set up a netty channel handler that does encryption using AES

There is no SASL required in the last step. The only reason it's involved in the last step now is because for DIGEST-MD5, the JDK mixes authentication and encryption. That is not the case with AES.

You're writing suboptimal code just because you're trying to support a non-feature. If you ignore SASL once authentication is done, you can write much simpler and better code for the AES path.

Copy link
Contributor Author

@chenjunjiedada chenjunjiedada Sep 29, 2016

Choose a reason for hiding this comment

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

I have removed the integrity check code it looks a bit simple now.

Also I am following your advice to setup netty channel handler for AES path and found SaslEncryption implement channel handler with lazy encryption and also it breaks incomming msg to chunks for transfer. So I'm concerning about whether we need to do same mechanism with Sasl encryption for netty channel handler, or just a simple implement that encrypt and decrypt the incoming msg. Could you please help to give some advice?

Copy link
Contributor

Choose a reason for hiding this comment

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

Lazy encryption is necessary, because the outbound pipeline is multi-threaded, and multiple threads might trigger the AES handler at the same time. So instead of encrypting in the handler method, you need to do like the SASL code and do it in a FileRegion implementation.

For AES, chunking should not be necessary. The SASL code chunks because it needs to know the size of each encrypted frame, to feed it to the unwrap method, otherwise decryption doesn't work. (Also to curb memory usage and avoid really large byte arrays.) AES should be able to decrypt the incoming bytes as they arrive, without the need for well-defined frames.

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 have some concern need your help to confirm.

  1. According to AES cipher API definition, it need to know the size of the encrypted data. I'm not sure whether AES decrypt incoming bytes without well-defined frames. Does receiver will receive exactly same message/ByteBuf from sender without decoding?
  2. It seems the new netty channel will look most same as SaslEncryption except the chunking you mentioned. So I think it would be better if we can reuse SaslEncryption here since chunking is also a optimization as it described. Could you please help to advice?

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure whether AES decrypt incoming bytes without well-defined frames.

That's easily figured out. Just write some code to try it out!

I actually specifically asked for test cases in Commons Crypto where the the size of blocks being encrypted and decrypted were different, because that should work when you're talking about decrypting a stream. You shouldn't need to frame things like you have to do with SASL.

So I think it would be better if we can reuse SaslEncryption

As I've said already, this is all Spark code and you can do whatever you think is best. I've given you guidelines to what I think the solution would look like. But I'm not gonna give you super-fine-grained instructions on how to achieve that, otherwise I'd just write the code myself.

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 have update the preliminary patch to use AES encryption steaming as suggested, could you please help to review it?

<tr>
<td><code>spark.authenticate.sasl.encryption.aes.enabled</code></td>
<td>false</td>
<td>
Copy link
Contributor

Choose a reason for hiding this comment

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

encryption

}

// Encrypt
byte[] encrypted = new byte[len + 10 + padding.length + 4];
Copy link
Contributor

Choose a reason for hiding this comment

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

Aside from the "why so much copying" and "why not use ByteBuffer" comments, there's a lot of magic numbers being used around this code...

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed magic numbers.

Actually, most of copies in wrap/unwrap are doing filling data not full copy, which should not cause any performance loss. Other copies also cannot be eliminated through ByteBuffer.

ByteBuffer response = client.sendRpcSync(buf.nioBuffer(), conf.saslRTTimeoutMs());
payload = saslClient.response(JavaUtils.bufferToArray(response));
}
boolean aesEnable = conf.saslEncryptionAesEnabled();
Copy link
Contributor

Choose a reason for hiding this comment

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

My initial idea is that this should be more dynamic... e.g. instead of an explicit configuration, if the user enables SASL encryption, the code would automatically negotiate the strongest possible cipher to use.

For example, the SaslMessage could have a new field saying whether the receiving side supports AES encryption, or which ciphers it supports. When talking to an old shuffle service, that field would be empty (on the wire), meaning it only supports built-in SASL ciphers. Newer servers would list AES-specific info there (e.g. enabled cipher transformations), and could use the new key negotiation code.

That's a little more work though. I like that it requires less input from the user. But maybe having an explicit option for the moment is fine...

Junjie Chen added 3 commits September 27, 2016 07:52
Change multiple encryption in wrap to one time encryption. Also refine code to remove unnecessary array copy.
Remove dependency statement which already exist in root pom.xml.
Add back dependency info, only remove version statement.
@SparkQA
Copy link

SparkQA commented Sep 27, 2016

Test build #65954 has finished for PR 15172 at commit 9eb31e2.

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

@SparkQA
Copy link

SparkQA commented Sep 27, 2016

Test build #65952 has finished for PR 15172 at commit 06b0009.

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

@SparkQA
Copy link

SparkQA commented Sep 27, 2016

Test build #65955 has finished for PR 15172 at commit 535945c.

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

Remove irrelevant integrity check.
@SparkQA
Copy link

SparkQA commented Sep 29, 2016

Test build #66080 has finished for PR 15172 at commit c1936eb.

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

@vanzin
Copy link
Contributor

vanzin commented Oct 3, 2016

To give you some pointers to what I think the AES path should look like:

  • decryption should work similarly to how CryptoInputStream works. It uses the Input interface which has a ReadableByteChannel implementation, but you don't really have a ReadableByteChannel here, just byte buffers. So you're gonna need some code to make those work together.
  • encryption: CryptoOutputStream has int write(ByteBuffer src) which should make it easy to implement the custom FileRegion used in Spark.

In both cases, there might be enhancements needed to the Commons Crypto library. That's something that we knew from the start. We should not provide a hacky implementation in Spark just because the features don't exist yet in Commons Crypto; this code has strict backwards compatibility requirements because it's used in the shuffle service, so we need a stable implementation from the start to avoid having to build version negotiation into the protocol.

@SparkQA
Copy link

SparkQA commented Oct 12, 2016

Test build #66785 has finished for PR 15172 at commit 0bf663f.

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

@SparkQA
Copy link

SparkQA commented Nov 8, 2016

Test build #68313 has finished for PR 15172 at commit 5ff19a5.

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

@SparkQA
Copy link

SparkQA commented Nov 8, 2016

Test build #68311 has finished for PR 15172 at commit 7071ca6.

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

Minor change to trigger another build, since build #68313 looks weird, it show fail unit test but details show it passed.
@chenjunjiedada
Copy link
Contributor Author

Build #68313 looks weird, details in link show it passed all test.

@SparkQA
Copy link

SparkQA commented Nov 8, 2016

Test build #68319 has finished for PR 15172 at commit c50e088.

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

// Generate a request config message to send to server.
AesConfigMessage configMessage = AesCipher.createConfigMessage(conf);
ByteBuffer buf = configMessage.encodeMessage();
client.sendRpcSync(buf, conf.saslRTTimeoutMs());
Copy link
Contributor

Choose a reason for hiding this comment

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

Wait. Isn't this now sending the keys in plain text over the wire?

I commented before that using 3des to encrypt the keys is sub-optimal, but I didn't mean for you to remove the encryption altogether.

client.sendRpcSync(buf, conf.saslRTTimeoutMs());
AesCipher cipher = new AesCipher(configMessage);
logger.info("Enabling AES cipher for client channel {}", client);
cipher.addToChannel(channel);
Copy link
Contributor

Choose a reason for hiding this comment

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

You need to dispose of the saslClient here.

Encrypt the config message since it was removed by mistake.
@chenjunjiedada
Copy link
Contributor Author

The encryption of config message was removed by mistake, very strange.. Add it back..

@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68434 has finished for PR 15172 at commit 573a8f3.

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

Junjie Chen added 2 commits November 10, 2016 10:51

// Encrypted the config message.
ByteBuffer encrypted = ByteBuffer.wrap(
saslClient.wrap(buf.array(), 0, buf.array().length));
Copy link
Contributor

Choose a reason for hiding this comment

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

JavaUtils.bufferToArray

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Really thanks:)

// Create AES cipher when it is authenticated
try {
byte[] encrypted;
if (message.hasArray()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

JavaUtils.bufferToArray here too?

@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68438 has finished for PR 15172 at commit cf7f62a.

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

refine code to use JavaUtils.bufferToArray.
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.

Finished my first round. Thanks! This is really cool!


private ByteBuffer currentEncrypted;

EncryptedMessage(CryptoOutputStream cos, Object msg, ByteArrayWritableChannel ch) {
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 document that cos will be redirected to ch.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ch is used to create the output encrypt stream, the document at L208 describe the reason to use two helper channel.

isComplete = true;
if (SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) {
if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) {
logger.debug("SASL authentication successful for channel {}", 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: this is a failure.

Copy link
Contributor

Choose a reason for hiding this comment

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

No, the message is correct. This means auth was successful and encryption (auth-conf) isn't being used.

@Override
public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception {
ByteBuf in = (ByteBuf) data;
byteChannel.feedData(in);
Copy link
Member

Choose a reason for hiding this comment

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

in should be retained.

Copy link
Member

Choose a reason for hiding this comment

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

Does Apache Common Crypto provide any API to decrypt ByteBuffer? If so, could you use it because the code will be much simpler (e.g., ByteArrayReadableChannel is totally not needed.)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Commons Crypto does provide ByteBuffer decryption in both openSslCipher and jceCipher, named update(ByteBuffer in, ByteBuffer out). But here we are using encryption/decryption streaming way to do the decryption.

import io.netty.buffer.ByteBuf;

public class ByteArrayReadableChannel implements ReadableByteChannel {
private final LinkedList<ByteBuf> buffers = new LinkedList<>();
Copy link
Member

Choose a reason for hiding this comment

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

Why not makeByteArrayReadableChannel be a wrapper for a single ByteBuf? Then the code will be much simpler. I don't see it's necessary to use LinkedList<ByteBuf>.


@Override
public int encodedLength() {
return 1 + 4 + ((inKey != null && inIv != null && outKey != null && outIv != null) ?
Copy link
Member

Choose a reason for hiding this comment

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

nit: don't allow null values then the codes will be simpler.

Copy link
Member

Choose a reason for hiding this comment

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

nit: since keySize is just inKey.length. You don't need to send it via the network.


int keySize = buf.readInt();

byte[] outKey = Encoders.ByteArrays.decode(buf);
Copy link
Member

Choose a reason for hiding this comment

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

why the decode order is not same as the encode order?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

client use inKey to encrypt, server use inKey to decrypt, So exchange the inkey and outkey here.

@Override
public int encodedLength() {
return 1 + 4 + ((inKey != null && inIv != null && outKey != null && outIv != null) ?
Encoders.ByteArrays.encodedLength(inKey) + Encoders.ByteArrays.encodedLength(inKey) +
Copy link
Member

Choose a reason for hiding this comment

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

nit: Encoders.ByteArrays.encodedLength(inKey) + Encoders.ByteArrays.encodedLength(outKey) +

byte[] outKey,
byte[] inIv,
byte[] outIv) throws IOException {
this.properties = properties;
Copy link
Member

Choose a reason for hiding this comment

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

This constructor is not used outside. Could you remove it?

@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68444 has finished for PR 15172 at commit 977bda7.

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

use ByteBuf directly to ease the implementation of ByteReadableChannel.
@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68467 has finished for PR 15172 at commit cb93e18.

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

// messages are being written to the channel while negotiation is still going on.
if (saslServer.isComplete()) {
if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) {
logger.debug("SASL authentication successful for channel {}", client);
Copy link
Contributor

Choose a reason for hiding this comment

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

Please restore this message.

@vanzin
Copy link
Contributor

vanzin commented Nov 10, 2016

LGTM, I'll leave it open a bit in case others want a second look.

@SparkQA
Copy link

SparkQA commented Nov 11, 2016

Test build #68499 has finished for PR 15172 at commit 6863efe.

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

@vanzin
Copy link
Contributor

vanzin commented Nov 11, 2016

retest this please

@SparkQA
Copy link

SparkQA commented Nov 11, 2016

Test build #68507 has finished for PR 15172 at commit 6863efe.

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

@vanzin
Copy link
Contributor

vanzin commented Nov 11, 2016

Merging to master.

@asfgit asfgit closed this in 4f15d94 Nov 11, 2016
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
## What changes were proposed in this pull request?

DIGEST-MD5 mechanism is used for SASL authentication and secure communication. DIGEST-MD5 mechanism supports 3DES, DES, and RC4 ciphers. However, 3DES, DES and RC4 are slow relatively.

AES provide better performance and security by design and is a replacement for 3DES according to NIST. Apache Common Crypto is a cryptographic library optimized with AES-NI, this patch employ Apache Common Crypto as enc/dec backend for SASL authentication and secure channel to improve spark RPC.
## How was this patch tested?

Unit tests and Integration test.

Author: Junjie Chen <junjie.j.chen@intel.com>

Closes apache#15172 from cjjnjust/shuffle_rpc_encrypt.
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