Skip to content

[SPARK-2938] Support SASL authentication in NettyBlockTransferService #3087

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

Closed
wants to merge 10 commits into from

Conversation

aarondav
Copy link
Contributor

@aarondav aarondav commented Nov 4, 2014

Also lays the groundwork for supporting it inside the external shuffle service.

Also lays the groundwork for supporting it inside the external shuffle service.
@aarondav
Copy link
Contributor Author

aarondav commented Nov 4, 2014

@tgravescs @andrewor14 @rxin PTAL

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22853 has started for PR 3087 at commit 8191bcb.

  • This patch merges cleanly.

@@ -150,7 +149,7 @@ private[spark] class BlockManager(
private val peerFetchLock = new Object
private var lastPeerFetchTime = 0L

initialize()
// initialize()
Copy link
Contributor

Choose a reason for hiding this comment

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

Are you planning to remove this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oops, yup.

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 another one at L110

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22853 has finished for PR 3087 at commit 8191bcb.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22853/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22862 has started for PR 3087 at commit 7b42adb.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22864 has started for PR 3087 at commit f6177d7.

  • This patch merges cleanly.

@aarondav
Copy link
Contributor Author

aarondav commented Nov 4, 2014

Alright, added unit tests and cleaned up docs and such, this is ready to go from my end.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22865/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22862 has finished for PR 3087 at commit 7b42adb.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22862/
Test PASSed.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22864 has finished for PR 3087 at commit f6177d7.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22864/
Test PASSed.


/** Encode a password as a base64-encoded char[] array. */
public static char[] encodePassword(String password) {
if (password != null) {
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 a change from the prior version -- before, only the client did this transformation, now both the server and client do.

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't follow your comment? It used to encode it for both client and server before as well.

It seems bad if you are trying to do a sasl negotiation with a null password.

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'm referring to this: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkSaslClient.scala#L118

I'm not sure what should be done about the case where both sides have a null password. Could throw exceptions on both sides.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22869 has started for PR 3087 at commit 785bbde.

  • This patch merges cleanly.

@@ -17,13 +17,17 @@

package org.apache.spark.network.netty

import org.apache.spark.network.util.TransportConf
Copy link
Contributor

Choose a reason for hiding this comment

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

import out of order here

@rxin
Copy link
Contributor

rxin commented Nov 4, 2014

This looks good pretty to me. @tgravescs it would be great if you can take a look since you wrote the original sasl code. Thanks.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22869 has finished for PR 3087 at commit 785bbde.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22869/
Test PASSed.

* initial state to the "authenticated" state. This client initializes the protocol via a
* firstToken, which is then followed by a set of challenges and responses.
*/
public class SparkSaslClient {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why was this (and other classes) changed to java?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For inclusion in the YARN node manager with minimal dependencies (no dependencies on Spark or Scala).

@tgravescs
Copy link
Contributor

Nothing major was jumping out at me. I haven't had time to look at the netty implementation in detail though either.

Did you do any manual testing? I would suggest creating some external process and just trying to hack the connection (pass bad password, auth off, etc) just as a full integration test.

@@ -55,4 +55,7 @@ public int connectionTimeoutMs() {

/** Send buffer size (SO_SNDBUF). */
public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); }

/** Timeout for a single round trip of SASL token exchange, in milliseconds. */
public int saslRTTimeout() { return conf.getInt("spark.shuffle.sasl.timeout", 30000); }
Copy link
Contributor

Choose a reason for hiding this comment

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

we should document this config

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Documentation for all network-related things is forthcoming soon, we don't have a place for it within this patch.

@aarondav
Copy link
Contributor Author

aarondav commented Nov 4, 2014

Addressed all non-documentation and testing related comments, I will work on those soon. For testing, I just need to add a NettyBlockTransfer-level integration test which matches the ConnectionManager's equivalent one.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22881 has started for PR 3087 at commit a6b95f1.

  • This patch merges cleanly.

@aarondav
Copy link
Contributor Author

aarondav commented Nov 4, 2014

@tgravescs @rxin Thanks for the review, I believe I've addressed all comments and I've added a full end-to-end test at the NettyBlockTransfer level.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22885 has started for PR 3087 at commit 3481718.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22881 has finished for PR 3087 at commit a6b95f1.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22881/
Test PASSed.

@rxin
Copy link
Contributor

rxin commented Nov 4, 2014

Merging this in master.

@tgravescs would be great if you continue looking at the new code. If you find new problems or have suggestions on better implementation/design, we will create new PRs to address them.

@rxin
Copy link
Contributor

rxin commented Nov 4, 2014

ASF git is down at the moment. Will merge when it comes back up.

* with the given appId. This appId allows a single SaslRpcHandler to multiplex different
* applications which may be using different sets of credentials.
*/
class SaslMessage implements Encodable {
Copy link
Contributor

Choose a reason for hiding this comment

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

Hey question, should this be fully public? Is it because there's no private[spark] equivalent in Java and we need to access it outside of its package?

Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if we should mark this @DeveloperAPI or something to prevent people from using it themselves.

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 actually package-private and is not accessible outside the package. In general, your point still stands, though, and there are other places where we are public because it must be accessed from a diff. package. We should have such an annotation.

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh yeah, sorry, this is package-private. I'm referring to SparkSaslServer and SparkSaslClient

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Gotcha, yeah, we need to figure out a scheme for @DeveloperAPI in this module....

@andrewor14
Copy link
Contributor

LGTM. If there are any outstanding issues we can fix them in subsequent patches.

@SparkQA
Copy link

SparkQA commented Nov 4, 2014

Test build #22885 has finished for PR 3087 at commit 3481718.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22885/
Test PASSed.

@rxin
Copy link
Contributor

rxin commented Nov 4, 2014

We shouldn't add developer api tag because this whole package should be private and not exposed to users. We should exclude this from all public docs.

@asfgit asfgit closed this in 5e73138 Nov 5, 2014
asfgit pushed a commit that referenced this pull request Nov 5, 2014
Also lays the groundwork for supporting it inside the external shuffle service.

Author: Aaron Davidson <[email protected]>

Closes #3087 from aarondav/sasl and squashes the following commits:

3481718 [Aaron Davidson] Delete rogue println
44f8410 [Aaron Davidson] Delete documentation - muahaha!
eb9f065 [Aaron Davidson] Improve documentation and add end-to-end test at Spark-level
a6b95f1 [Aaron Davidson] Address comments
785bbde [Aaron Davidson] Cleanup
79973cb [Aaron Davidson] Remove unused file
151b3c5 [Aaron Davidson] Add docs, timeout config, better failure handling
f6177d7 [Aaron Davidson] Cleanup SASL state upon connection termination
7b42adb [Aaron Davidson] Add unit tests
8191bcb [Aaron Davidson] [SPARK-2938] Support SASL authentication in NettyBlockTransferService
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