-
Couldn't load subscription status.
- Fork 28.9k
[SPARK-9782] [YARN] Support YARN application tags via SparkConf #8072
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -163,6 +163,23 @@ private[spark] class Client( | |
| appContext.setQueue(args.amQueue) | ||
| appContext.setAMContainerSpec(containerContext) | ||
| appContext.setApplicationType("SPARK") | ||
| sparkConf.getOption(CONF_SPARK_YARN_APPLICATION_TAGS) | ||
| .map(StringUtils.getTrimmedStringCollection(_)) | ||
| .filter(!_.isEmpty()) | ||
| .foreach { tagCollection => | ||
| try { | ||
| // The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use | ||
| // reflection to set it, printing a warning if a tag was specified but the YARN version | ||
| // doesn't support it. | ||
| val method = appContext.getClass().getMethod( | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a comment here mentioning why this needs to be called via reflection There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
| "setApplicationTags", classOf[java.util.Set[String]]) | ||
| method.invoke(appContext, new java.util.HashSet[String](tagCollection)) | ||
| } catch { | ||
| case e: NoSuchMethodException => | ||
| logWarning(s"Ignoring $CONF_SPARK_YARN_APPLICATION_TAGS because this version of " + | ||
| "YARN does not support it") | ||
| } | ||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. using reflect make the code reading more difficult. generally not be recommanded. |
||
| sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt) match { | ||
| case Some(v) => appContext.setMaxAppAttempts(v) | ||
| case None => logDebug("spark.yarn.maxAppAttempts is not set. " + | ||
|
|
@@ -982,6 +999,10 @@ object Client extends Logging { | |
| // of the executors | ||
| val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" | ||
|
|
||
| // Comma-separated list of strings to pass through as YARN application tags appearing | ||
| // in YARN ApplicationReports, which can be used for filtering when querying YARN. | ||
| val CONF_SPARK_YARN_APPLICATION_TAGS = "spark.yarn.tags" | ||
|
|
||
| // Staging directory is private! -> rwx-------- | ||
| val STAGING_DIR_PERMISSION: FsPermission = | ||
| FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,8 +29,11 @@ import org.apache.hadoop.conf.Configuration | |
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.mapreduce.MRJobConfig | ||
| import org.apache.hadoop.yarn.api.ApplicationConstants.Environment | ||
| import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse | ||
| import org.apache.hadoop.yarn.api.records._ | ||
| import org.apache.hadoop.yarn.client.api.YarnClientApplication | ||
| import org.apache.hadoop.yarn.conf.YarnConfiguration | ||
| import org.apache.hadoop.yarn.util.Records | ||
| import org.mockito.Matchers._ | ||
| import org.mockito.Mockito._ | ||
| import org.scalatest.{BeforeAndAfterAll, Matchers} | ||
|
|
@@ -170,6 +173,39 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll { | |
| cp should contain ("/remotePath/my1.jar") | ||
| } | ||
|
|
||
| test("configuration and args propagate through createApplicationSubmissionContext") { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Have you verified that this test passes both against versions of YARN that support app tags and against versions that do not? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Indeed I did before sending out the first PR: $ sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver yarn/test test-only $ sbt/sbt -Pyarn -Phadoop-2.4 -Phive -Phive-thriftserver yarn/test Kicking off a new round of build/tests locally now after the changes. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Tests were green with both a 2.3 and 2.4 build. Thanks for the help reviewing! There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Console output from my test runs here: https://gist.github.com/dennishuo/79212ba4f8a3bfb71227 |
||
| val conf = new Configuration() | ||
| // When parsing tags, duplicates and leading/trailing whitespace should be removed. | ||
| // Spaces between non-comma strings should be preserved as single tags. Empty strings may or | ||
| // may not be removed depending on the version of Hadoop being used. | ||
| val sparkConf = new SparkConf() | ||
| .set(Client.CONF_SPARK_YARN_APPLICATION_TAGS, ",tag1, dup,tag2 , ,multi word , dup") | ||
| .set("spark.yarn.maxAppAttempts", "42") | ||
| val args = new ClientArguments(Array( | ||
| "--name", "foo-test-app", | ||
| "--queue", "staging-queue"), sparkConf) | ||
|
|
||
| val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) | ||
| val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) | ||
| val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) | ||
|
|
||
| val client = new Client(args, conf, sparkConf) | ||
| client.createApplicationSubmissionContext( | ||
| new YarnClientApplication(getNewApplicationResponse, appContext), | ||
| containerLaunchContext) | ||
|
|
||
| appContext.getApplicationName should be ("foo-test-app") | ||
| appContext.getQueue should be ("staging-queue") | ||
| appContext.getAMContainerSpec should be (containerLaunchContext) | ||
| appContext.getApplicationType should be ("SPARK") | ||
| appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method => | ||
| val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]] | ||
| tags should contain allOf ("tag1", "dup", "tag2", "multi word") | ||
| tags.filter(!_.isEmpty).size should be (4) | ||
| } | ||
| appContext.getMaxAppAttempts should be (42) | ||
| } | ||
|
|
||
| object Fixtures { | ||
|
|
||
| val knownDefYarnAppCP: Seq[String] = | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not a big deal, but I'd flip this around and do something like:
That way you print the log message only once, lookup the method only once, and skip all the processing of the option if the method is not available.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BTW if you do this you're probably better off doing:
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the suggestion! I'm a bit rusty in Scala, so I could be misapplying some idioms here, but my intent on this line was to use the foreach, etc., only to access the "None or single element" of the Option, as indicated on http://www.scala-lang.org/api/current/index.html#scala.Option , so the "foreach" here is supposed to only iterate over the "single" element which happens to itself be a collection, as opposed to iterating over elements of the inner collection. So there shouldn't be any way to cause multiple log statements or multiple reflection-based lookups of the method.
I also wanted to err on the side of minimizing behavioral changes for existing setups, so that if the Option is None, then this chaining as a monad avoids ever needing to lookup a method, or even invoking any of the option-processing methods like StringUtils.getTrimmedStringCollection.
I could add a note to make it more clear that the map/filter/foreach is on the Option, as opposed to the Collection if that'd help.
Anyhow, I'll be happy to apply the reversal to start with the method as you suggest if you prefer, just wanted to hear your thoughts on this Option usage first.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, I see.
setApplicationTagstakes aSet[String], not a single string, so you're not calling it once per tag in the collection. My bad. Current code is fine.