Skip to content

Commit d6fb485

Browse files
jerryshaoMarcelo Vanzin
authored andcommitted
[SPARK-14423][YARN] Avoid same name files added to distributed cache again
## What changes were proposed in this pull request? In the current implementation of assembly-free spark deployment, jars under `assembly/target/scala-xxx/jars` will be uploaded to distributed cache by default, there's a chance these jars' name will be conflicted with name of jars specified in `--jars`, this will introduce exception when starting application: ``` client token: N/A diagnostics: Application application_1459907402325_0004 failed 2 times due to AM Container for appattempt_1459907402325_0004_000002 exited with exitCode: -1000 For more detailed output, check application tracking page:http://hw12100.local:8088/proxy/application_1459907402325_0004/Then, click on links to logs of each attempt. Diagnostics: Resource hdfs://localhost:8020/user/sshao/.sparkStaging/application_1459907402325_0004/avro-mapred-1.7.7-hadoop2.jar changed on src filesystem (expected 1459909780508, was 1459909782590 java.io.IOException: Resource hdfs://localhost:8020/user/sshao/.sparkStaging/application_1459907402325_0004/avro-mapred-1.7.7-hadoop2.jar changed on src filesystem (expected 1459909780508, was 1459909782590 at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:253) at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:61) at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:359) at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:357) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:356) at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:60) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` So here by checking the name of file to avoid same name files uploaded again. ## How was this patch tested? Unit test and manual integrated test is done locally. Author: jerryshao <[email protected]> Closes #12203 from jerryshao/SPARK-14423.
1 parent 1a39664 commit d6fb485

File tree

2 files changed

+42
-4
lines changed

2 files changed

+42
-4
lines changed

yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -364,6 +364,10 @@ private[spark] class Client(
364364
// multiple times, YARN will fail to launch containers for the app with an internal
365365
// error.
366366
val distributedUris = new HashSet[String]
367+
// Used to keep track of URIs(files) added to the distribute cache have the same name. If
368+
// same name but different path files are added multiple time, YARN will fail to launch
369+
// containers for the app with an internal error.
370+
val distributedNames = new HashSet[String]
367371
YarnSparkHadoopUtil.get.obtainTokenForHiveMetastore(sparkConf, hadoopConf, credentials)
368372
YarnSparkHadoopUtil.get.obtainTokenForHBase(sparkConf, hadoopConf, credentials)
369373

@@ -376,11 +380,16 @@ private[spark] class Client(
376380

377381
def addDistributedUri(uri: URI): Boolean = {
378382
val uriStr = uri.toString()
383+
val fileName = new File(uri.getPath).getName
379384
if (distributedUris.contains(uriStr)) {
380-
logWarning(s"Resource $uri added multiple times to distributed cache.")
385+
logWarning(s"Same path resource $uri added multiple times to distributed cache.")
386+
false
387+
} else if (distributedNames.contains(fileName)) {
388+
logWarning(s"Same name resource $uri added multiple times to distributed cache")
381389
false
382390
} else {
383391
distributedUris += uriStr
392+
distributedNames += fileName
384393
true
385394
}
386395
}
@@ -519,8 +528,7 @@ private[spark] class Client(
519528
).foreach { case (flist, resType, addToClasspath) =>
520529
flist.foreach { file =>
521530
val (_, localizedPath) = distribute(file, resType = resType)
522-
require(localizedPath != null)
523-
if (addToClasspath) {
531+
if (addToClasspath && localizedPath != null) {
524532
cachedSecondaryJarLinks += localizedPath
525533
}
526534
}

yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala

Lines changed: 31 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.deploy.yarn
1919

20-
import java.io.{File, FileOutputStream}
20+
import java.io.{File, FileInputStream, FileOutputStream}
2121
import java.net.URI
2222
import java.util.Properties
2323

@@ -285,6 +285,36 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll
285285
classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
286286
}
287287

288+
test("ignore same name jars") {
289+
val libs = Utils.createTempDir()
290+
val jarsDir = new File(libs, "jars")
291+
assert(jarsDir.mkdir())
292+
new FileOutputStream(new File(libs, "RELEASE")).close()
293+
val userLibs = Utils.createTempDir()
294+
295+
val jar1 = TestUtils.createJarWithFiles(Map(), jarsDir)
296+
val jar2 = TestUtils.createJarWithFiles(Map(), userLibs)
297+
// Copy jar2 to jar3 with same name
298+
val jar3 = {
299+
val target = new File(userLibs, new File(jar1.toURI).getName)
300+
val input = new FileInputStream(jar2.getPath)
301+
val output = new FileOutputStream(target)
302+
Utils.copyStream(input, output, closeStreams = true)
303+
target.toURI.toURL
304+
}
305+
306+
val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
307+
.set(JARS_TO_DISTRIBUTE, Seq(jar2.getPath, jar3.getPath))
308+
309+
val client = createClient(sparkConf)
310+
val tempDir = Utils.createTempDir()
311+
client.prepareLocalResources(tempDir.getAbsolutePath(), Nil)
312+
313+
// Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar1 will be
314+
// ignored.
315+
sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName)))
316+
}
317+
288318
object Fixtures {
289319

290320
val knownDefYarnAppCP: Seq[String] =

0 commit comments

Comments
 (0)