From b9d2ced92a9712edf7612f4cee1ac929e5fa3dab Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 28 Jul 2015 09:17:15 -0500 Subject: [PATCH 01/43] incomplete setup for external shuffle service tests --- yarn/pom.xml | 6 + .../deploy/yarn/ExternalShuffleSuite.scala | 305 ++++++++++++++++++ 2 files changed, 311 insertions(+) create mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala diff --git a/yarn/pom.xml b/yarn/pom.xml index 2aeed98285aa8..11a88c5e08486 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -39,6 +39,12 @@ spark-core_${scala.binary.version} ${project.version} + + org.apache.spark + spark-network-yarn_${scala.binary.version} + ${project.version} + test + org.apache.spark spark-core_${scala.binary.version} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala new file mode 100644 index 0000000000000..f5a45e6256f26 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala @@ -0,0 +1,305 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.deploy.yarn + +import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.net.URL +import java.util.Properties +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.ByteStreams +import com.google.common.io.Files +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster +import org.scalatest.{BeforeAndAfterAll, Matchers} + +import org.apache.spark._ +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, + SparkListenerExecutorAdded} +import org.apache.spark.util.Utils + +/** +* Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN +* applications, and require the Spark assembly to be built before they can be successfully +* run. +*/ +class ExternalShuffleSuite extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { + + // log4j configuration for the YARN containers, so that their output is collected + // by YARN instead of trying to overwrite unit-tests.log. + private val LOG4J_CONF = """ + |log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + """.stripMargin + + private val TEST_PYFILE = """ + |import mod1, mod2 + |import sys + |from operator import add + | + |from pyspark import SparkConf , SparkContext + |if __name__ == "__main__": + | if len(sys.argv) != 2: + | print >> sys.stderr, "Usage: test.py [result file]" + | exit(-1) + | sc = SparkContext(conf=SparkConf()) + | status = open(sys.argv[1],'w') + | result = "failure" + | rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func()) + | cnt = rdd.count() + | if cnt == 10: + | result = "success" + | status.write(result) + | status.close() + | sc.stop() + """.stripMargin + + private val TEST_PYMODULE = """ + |def func(): + | return 42 + """.stripMargin + + private var yarnCluster: MiniYARNCluster = _ + private var tempDir: File = _ + private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ + private var logConfDir: File = _ + + override def beforeAll() { + super.beforeAll() + + tempDir = Utils.createTempDir() + logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + System.setProperty("SPARK_YARN_MODE", "true") + + val logConfFile = new File(logConfDir, "log4j.properties") + Files.write(LOG4J_CONF, logConfFile, UTF_8) + + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + val yarnConfig = new YarnConfiguration() + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), + "org.apache.spark.network.yarn.YarnShuffleService") + yarnCluster.init(yarnConfig) + yarnCluster.start() + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + + fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) + } + + override def afterAll() { + yarnCluster.stop() + System.clearProperty("SPARK_YARN_MODE") + super.afterAll() + } + + test("external shuffle service") { + val result = File.createTempFile("result", null, tempDir) + runSpark( + false, + mainClassName(ExternalShuffleDriver.getClass), + appArgs = Seq(result.getAbsolutePath()), + extraConf = Map() +// extraConf = Map("spark.shuffle.service.enabled" -> "true") + ) + checkResult(result) + } + + private def runSpark( + clientMode: Boolean, + klass: String, + appArgs: Seq[String] = Nil, + sparkArgs: Seq[String] = Nil, + extraClassPath: Seq[String] = Nil, + extraJars: Seq[String] = Nil, + extraConf: Map[String, String] = Map()): Unit = { + val master = if (clientMode) "yarn-client" else "yarn-cluster" + val props = new Properties() + + props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) + + val childClasspath = logConfDir.getAbsolutePath() + + File.pathSeparator + + sys.props("java.class.path") + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator) + props.setProperty("spark.driver.extraClassPath", childClasspath) + props.setProperty("spark.executor.extraClassPath", childClasspath) + + // SPARK-4267: make sure java options are propagated correctly. + props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") + props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") + + yarnCluster.getConfig().foreach { e => + props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) + } + + sys.props.foreach { case (k, v) => + if (k.startsWith("spark.")) { + props.setProperty(k, v) + } + } + + extraConf.foreach { case (k, v) => props.setProperty(k, v) } + + val propsFile = File.createTempFile("spark", ".properties", tempDir) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + props.store(writer, "Spark properties.") + writer.close() + + val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil + val mainArgs = + if (klass.endsWith(".py")) { + Seq(klass) + } else { + Seq("--class", klass, fakeSparkJar.getAbsolutePath()) + } + val argv = + Seq( + new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), + "--master", master, + "--num-executors", "1", + "--properties-file", propsFile.getAbsolutePath()) ++ + extraJarArgs ++ + sparkArgs ++ + mainArgs ++ + appArgs + + Utils.executeAndGetOutput(argv, + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) + } + + /** + * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide + * any sort of error when the job process finishes successfully, but the job itself fails. So + * the tests enforce that something is written to a file after everything is ok to indicate + * that the job succeeded. + */ + private def checkResult(result: File): Unit = { + checkResult(result, "success") + } + + private def checkResult(result: File, expected: String): Unit = { + var resultString = Files.toString(result, UTF_8) + resultString should be (expected) + } + + private def mainClassName(klass: Class[_]): String = { + klass.getName().stripSuffix("$") + } + +} + +private object ExternalShuffleDriver extends Logging with Matchers { + + val WAIT_TIMEOUT_MILLIS = 10000 + + def main(args: Array[String]): Unit = { + if (args.length != 1) { + // scalastyle:off println + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: ExternalShuffleDriver [result file] + """.stripMargin) + // scalastyle:on println + System.exit(1) + } + + val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) + .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) + val conf = sc.getConf + val status = new File(args(0)) + var result = "failure" + try { + val data = sc.parallelize(1 to 4, 4).collect().toSet + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + data should be (Set(1, 2, 3, 4)) + result = "success" + } finally { + sc.stop() + Files.write(result, status, UTF_8) + } + + // verify log urls are present + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => + assert(info.logUrlMap.nonEmpty) + } + + // If we are running in yarn-cluster mode, verify that driver logs links and present and are + // in the expected format. + if (conf.get("spark.master") == "yarn-cluster") { + assert(listener.driverLogs.nonEmpty) + val driverLogs = listener.driverLogs.get + assert(driverLogs.size === 2) + assert(driverLogs.containsKey("stderr")) + assert(driverLogs.containsKey("stdout")) + val urlStr = driverLogs("stderr") + // Ensure that this is a valid URL, else this will throw an exception + new URL(urlStr) + val containerId = YarnSparkHadoopUtil.get.getContainerId + val user = Utils.getCurrentUserName() + assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) + } + } + +} From 36127d38750e75d682c8858b7128bd634732be71 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 28 Jul 2015 13:27:02 -0500 Subject: [PATCH 02/43] wip --- .../scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala | 1 + yarn/src/test/resources/log4j.properties | 2 +- .../org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 78e27fb7f3337..6497babc7262d 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -105,6 +105,7 @@ class ExecutorRunnable( // Authentication is not enabled, so just provide dummy metadata ByteBuffer.allocate(0) } + //throw an exception here, and things just hang ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) } diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index 6b8a5dbf6373e..1eb7ca41859e6 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -25,4 +25,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.apache.hadoop=INFO diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala index f5a45e6256f26..3212bdffd0e19 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala @@ -151,8 +151,8 @@ class ExternalShuffleSuite extends SparkFunSuite with BeforeAndAfterAll with Mat false, mainClassName(ExternalShuffleDriver.getClass), appArgs = Seq(result.getAbsolutePath()), - extraConf = Map() -// extraConf = Map("spark.shuffle.service.enabled" -> "true") +// extraConf = Map() + extraConf = Map("spark.shuffle.service.enabled" -> "true") ) checkResult(result) } From bb3ba494be05a11ab8105ce4ce570f60d1d33e51 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 28 Jul 2015 13:27:12 -0500 Subject: [PATCH 03/43] minor cleanup --- .../org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala | 2 +- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala index 3212bdffd0e19..28d31c7cc4525 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala @@ -232,7 +232,7 @@ class ExternalShuffleSuite extends SparkFunSuite with BeforeAndAfterAll with Mat } private def checkResult(result: File, expected: String): Unit = { - var resultString = Files.toString(result, UTF_8) + val resultString = Files.toString(result, UTF_8) resultString should be (expected) } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 547863d9a0739..15021eccafc93 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -174,7 +174,7 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher } private def testBasicYarnApp(clientMode: Boolean): Unit = { - var result = File.createTempFile("result", null, tempDir) + val result = File.createTempFile("result", null, tempDir) runSpark(clientMode, mainClassName(YarnClusterDriver.getClass), appArgs = Seq(result.getAbsolutePath())) checkResult(result) @@ -299,7 +299,7 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher } private def checkResult(result: File, expected: String): Unit = { - var resultString = Files.toString(result, UTF_8) + val resultString = Files.toString(result, UTF_8) resultString should be (expected) } From c69f46b98b6b9da16bfc7fc3a15deeade3e229ed Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 30 Jul 2015 15:08:49 -0500 Subject: [PATCH 04/43] maybe working version, needs tests & cleanup ... --- .../spark/deploy/ExternalShuffleService.scala | 2 +- .../spark/ExternalShuffleServiceSuite.scala | 2 +- .../shuffle/ExternalShuffleBlockHandler.java | 8 ++- .../shuffle/ExternalShuffleBlockResolver.java | 63 ++++++++++++++++--- .../shuffle/protocol/ExecutorShuffleInfo.java | 3 +- .../network/yarn/YarnShuffleService.java | 29 +++++++-- 6 files changed, 89 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index 09973a0a2c998..68ef2a28ee311 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -45,7 +45,7 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana private val useSasl: Boolean = securityManager.isAuthenticationEnabled() private val transportConf = SparkTransportConf.fromSparkConf(sparkConf, numUsableCores = 0) - private val blockHandler = new ExternalShuffleBlockHandler(transportConf) + private val blockHandler = new ExternalShuffleBlockHandler(transportConf, null) private val transportContext: TransportContext = new TransportContext(transportConf, blockHandler) private var server: TransportServer = _ diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index c38d70252add1..e846a72c888c6 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -36,7 +36,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { override def beforeAll() { val transportConf = SparkTransportConf.fromSparkConf(conf, numUsableCores = 2) - rpcHandler = new ExternalShuffleBlockHandler(transportConf) + rpcHandler = new ExternalShuffleBlockHandler(transportConf, null) val transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index e4faaf8854fc7..a4f1b27a658c1 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -17,6 +17,8 @@ package org.apache.spark.network.shuffle; +import java.io.File; +import java.io.IOException; import java.util.List; import com.google.common.annotations.VisibleForTesting; @@ -49,8 +51,10 @@ public class ExternalShuffleBlockHandler extends RpcHandler { private final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler(TransportConf conf) { - this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf)); + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) + throws IOException, ClassNotFoundException { + this(new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } /** Enables mocking out the StreamManager and BlockManager. */ diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 022ed88a16480..8a6a790905b22 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -17,10 +17,7 @@ package org.apache.spark.network.shuffle; -import java.io.DataInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; +import java.io.*; import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -60,18 +57,26 @@ public class ExternalShuffleBlockResolver { private final TransportConf conf; - public ExternalShuffleBlockResolver(TransportConf conf) { - this(conf, Executors.newSingleThreadExecutor( + private final File registeredExecutorFile; + + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) + throws IOException, ClassNotFoundException { + this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); } // Allows tests to have more control over when directories are cleaned up. @VisibleForTesting - ExternalShuffleBlockResolver(TransportConf conf, Executor directoryCleaner) { + ExternalShuffleBlockResolver( + TransportConf conf, + File registeredExecutorFile, + Executor directoryCleaner) throws IOException, ClassNotFoundException { this.conf = conf; + this.registeredExecutorFile = registeredExecutorFile; this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; + reloadRegisteredExecutors(); } /** Registers a new Executor with all the configuration we need to find its shuffle files. */ @@ -81,7 +86,14 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); - executors.put(fullId, executorInfo); + synchronized (executors) { + executors.put(fullId, executorInfo); + try { + saveRegisteredExecutors(); + } catch (Exception e) { + logger.error("Error saving registered executors", e); + } + } } /** @@ -221,7 +233,7 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) } /** Simply encodes an executor's full ID, which is appId + execId. */ - private static class AppExecId { + private static class AppExecId implements Serializable { final String appId; final String execId; @@ -252,4 +264,37 @@ public String toString() { .toString(); } } + + private void reloadRegisteredExecutors() throws IOException, ClassNotFoundException { + if (registeredExecutorFile != null && registeredExecutorFile.exists()) { + ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); + int nExecutors = in.readInt(); + logger.info("Reloading executors from {}", registeredExecutorFile); + for (int i = 0; i < nExecutors; i++) { + AppExecId appExecId = (AppExecId) in.readObject(); + ExecutorShuffleInfo shuffleInfo = (ExecutorShuffleInfo) in.readObject(); + logger.info("Reregistering executor {} with {}", appExecId, shuffleInfo); + executors.put(appExecId, shuffleInfo); + } + in.close(); + } else { + logger.info("No executor info to reload"); + } + } + + private void saveRegisteredExecutors() throws IOException { + if (registeredExecutorFile != null) { + logger.info("Saving registered executors to {}", registeredExecutorFile); + ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(registeredExecutorFile)); + // synchronize so we can write out the size :( + synchronized (executors) { + out.writeInt(executors.size()); + for (Map.Entry e: executors.entrySet()) { + out.writeObject(e.getKey()); + out.writeObject(e.getValue()); + } + } + out.close(); + } + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index cadc8e8369c6a..71b45dd5c80cf 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -17,6 +17,7 @@ package org.apache.spark.network.shuffle.protocol; +import java.io.Serializable; import java.util.Arrays; import com.google.common.base.Objects; @@ -26,7 +27,7 @@ import org.apache.spark.network.protocol.Encoders; /** Contains all configuration necessary for locating the shuffle files of an executor. */ -public class ExecutorShuffleInfo implements Encodable { +public class ExecutorShuffleInfo implements Encodable, Serializable { /** The base set of local directories that the executor stores its shuffle files in. */ public final String[] localDirs; /** Number of subdirectories created within each localDir. */ diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 463f99ef3352d..5efdb63c2af1d 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -17,12 +17,13 @@ package org.apache.spark.network.yarn; +import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.AuxiliaryService; import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; @@ -35,7 +36,6 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.sasl.SaslServerBootstrap; import org.apache.spark.network.sasl.ShuffleSecretManager; -import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; @@ -81,6 +81,8 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks private ExternalShuffleBlockHandler blockHandler; + private File registeredExecutorFile; + public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); @@ -100,11 +102,19 @@ private boolean isAuthenticationEnabled() { */ @Override protected void serviceInit(Configuration conf) { + + registeredExecutorFile = + findRegisteredExecutorFile(conf.get("yarn.nodemanager.local-dirs").split(",")); + TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); - blockHandler = new ExternalShuffleBlockHandler(transportConf); + try { + blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); + } catch (Exception e) { + logger.error("Failed to initial external shuffle service", e); + } List bootstraps = Lists.newArrayList(); if (authEnabled) { @@ -118,7 +128,8 @@ protected void serviceInit(Configuration conf) { shuffleServer = transportContext.createServer(port, bootstraps); String authEnabledString = authEnabled ? "enabled" : "not enabled"; logger.info("Started YARN shuffle service for Spark on port {}. " + - "Authentication is {}.", port, authEnabledString); + "Authentication is {}. Registered executor file is {}", port, authEnabledString, + registeredExecutorFile); } @Override @@ -161,6 +172,16 @@ public void stopContainer(ContainerTerminationContext context) { logger.info("Stopping container {}", containerId); } + private File findRegisteredExecutorFile(String[] localDirs) { + for (String dir: localDirs) { + File f = new File(dir, "registeredExecutors.bin"); + if (f.exists()) { + return f; + } + } + return new File(localDirs[0], "registeredExecutors.bin"); + } + /** * Close the shuffle server to clean up any associated state. */ From 5e5a7c36c3f6ac2c0f0dd6738eb4bb66605f8428 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 10:37:14 -0500 Subject: [PATCH 05/43] fix build --- .../shuffle/ExternalShuffleBlockResolverSuite.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index d02f4f0fdb682..49509a201f1c4 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -59,8 +59,8 @@ public static void afterAll() { } @Test - public void testBadRequests() { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + public void testBadRequests() throws IOException, ClassNotFoundException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); @@ -90,8 +90,8 @@ public void testBadRequests() { } @Test - public void testSortShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + public void testSortShuffleBlocks() throws IOException, ClassNotFoundException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); @@ -109,8 +109,8 @@ public void testSortShuffleBlocks() throws IOException { } @Test - public void testHashShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + public void testHashShuffleBlocks() throws IOException, ClassNotFoundException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); From ad122ef6b07eba0326929057704ef827a7eb26d6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 10:40:00 -0500 Subject: [PATCH 06/43] more fixes --- .../shuffle/ExternalShuffleCleanupSuite.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index d9d9c1bf2f17a..2986c15ff97d5 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -38,11 +38,11 @@ public class ExternalShuffleCleanupSuite { TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); @Test - public void noCleanupAndCleanup() throws IOException { + public void noCleanupAndCleanup() throws IOException, ClassNotFoundException { TestShuffleDataContext dataContext = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); resolver.applicationRemoved("app", false /* cleanup */); @@ -55,7 +55,7 @@ public void noCleanupAndCleanup() throws IOException { } @Test - public void cleanupUsesExecutor() throws IOException { + public void cleanupUsesExecutor() throws IOException, ClassNotFoundException { TestShuffleDataContext dataContext = createSomeData(); final AtomicBoolean cleanupCalled = new AtomicBoolean(false); @@ -65,7 +65,8 @@ public void cleanupUsesExecutor() throws IOException { @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } }; - ExternalShuffleBlockResolver manager = new ExternalShuffleBlockResolver(conf, noThreadExecutor); + ExternalShuffleBlockResolver manager = + new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); manager.applicationRemoved("app", true); @@ -78,12 +79,12 @@ public void cleanupUsesExecutor() throws IOException { } @Test - public void cleanupMultipleExecutors() throws IOException { + public void cleanupMultipleExecutors() throws IOException, ClassNotFoundException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); @@ -94,12 +95,12 @@ public void cleanupMultipleExecutors() throws IOException { } @Test - public void cleanupOnlyRemovedApp() throws IOException { + public void cleanupOnlyRemovedApp() throws IOException, ClassNotFoundException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); resolver.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); From 0b588bd5e0dd4ee81f9b4f856f9a57f4a4c72151 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 10:52:57 -0500 Subject: [PATCH 07/43] more fixes ... --- .../network/shuffle/ExternalShuffleIntegrationSuite.java | 4 ++-- .../spark/network/shuffle/ExternalShuffleSecuritySuite.java | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 39aa49911d9cb..c2de8d2111f0e 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -73,7 +73,7 @@ public class ExternalShuffleIntegrationSuite { }; @BeforeClass - public static void beforeAll() throws IOException { + public static void beforeAll() throws IOException, ClassNotFoundException { Random rand = new Random(); for (byte[] block : exec0Blocks) { @@ -92,7 +92,7 @@ public static void beforeAll() throws IOException { dataContext1.insertHashShuffleData(1, 0, exec1Blocks); conf = new TransportConf(new SystemPropertyConfigProvider()); - handler = new ExternalShuffleBlockHandler(conf); + handler = new ExternalShuffleBlockHandler(conf, null); TransportContext transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index d4ec1956c1e29..d57b65ca429a6 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -43,8 +43,9 @@ public class ExternalShuffleSecuritySuite { TransportServer server; @Before - public void beforeEach() { - TransportContext context = new TransportContext(conf, new ExternalShuffleBlockHandler(conf)); + public void beforeEach() throws IOException, ClassNotFoundException { + TransportContext context = + new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, new TestSecretKeyHolder("my-app-id", "secret")); this.server = context.createServer(Arrays.asList(bootstrap)); From 1136f44eb9bca70c365db8e004d7b84840f9dc55 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 15:33:56 -0500 Subject: [PATCH 08/43] test needs to have an actual shuffle --- .../deploy/yarn/ExternalShuffleSuite.scala | 29 ++----------------- 1 file changed, 2 insertions(+), 27 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala index 28d31c7cc4525..63cae5370bc1b 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala @@ -266,7 +266,8 @@ private object ExternalShuffleDriver extends Logging with Matchers { val status = new File(args(0)) var result = "failure" try { - val data = sc.parallelize(1 to 4, 4).collect().toSet + val data = sc.parallelize(1 to 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. + collect().toSet sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) data should be (Set(1, 2, 3, 4)) result = "success" @@ -274,32 +275,6 @@ private object ExternalShuffleDriver extends Logging with Matchers { sc.stop() Files.write(result, status, UTF_8) } - - // verify log urls are present - val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] - assert(listeners.size === 1) - val listener = listeners(0) - val executorInfos = listener.addedExecutorInfos.values - assert(executorInfos.nonEmpty) - executorInfos.foreach { info => - assert(info.logUrlMap.nonEmpty) - } - - // If we are running in yarn-cluster mode, verify that driver logs links and present and are - // in the expected format. - if (conf.get("spark.master") == "yarn-cluster") { - assert(listener.driverLogs.nonEmpty) - val driverLogs = listener.driverLogs.get - assert(driverLogs.size === 2) - assert(driverLogs.containsKey("stderr")) - assert(driverLogs.containsKey("stdout")) - val urlStr = driverLogs("stderr") - // Ensure that this is a valid URL, else this will throw an exception - new URL(urlStr) - val containerId = YarnSparkHadoopUtil.get.getContainerId - val user = Utils.getCurrentUserName() - assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) - } } } From 9eae11939a408639ec990b36407b1a330d20961a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 15:43:44 -0500 Subject: [PATCH 09/43] cleanup lots of duplication --- .../deploy/yarn/BaseYarnClusterSuite.scala | 199 ++++++++++++++++++ .../deploy/yarn/ExternalShuffleSuite.scala | 196 +---------------- .../spark/deploy/yarn/YarnClusterSuite.scala | 156 +------------- 3 files changed, 210 insertions(+), 341 deletions(-) create mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala new file mode 100644 index 0000000000000..d0161f8efe3ba --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.net.URL +import java.util.Properties +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.ByteStreams +import com.google.common.io.Files +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster +import org.scalatest.{BeforeAndAfterAll, Matchers} + +import org.apache.spark._ +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, + SparkListenerExecutorAdded} +import org.apache.spark.util.Utils + +abstract class BaseYarnClusterSuite + extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { + + // log4j configuration for the YARN containers, so that their output is collected + // by YARN instead of trying to overwrite unit-tests.log. + protected val LOG4J_CONF = """ + |log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + """.stripMargin + + private var yarnCluster: MiniYARNCluster = _ + protected var tempDir: File = _ + private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ + private var logConfDir: File = _ + + + def yarnConfig: YarnConfiguration + + override def beforeAll() { + super.beforeAll() + + tempDir = Utils.createTempDir() + logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + System.setProperty("SPARK_YARN_MODE", "true") + + val logConfFile = new File(logConfDir, "log4j.properties") + Files.write(LOG4J_CONF, logConfFile, UTF_8) + + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + yarnCluster.init(yarnConfig) + yarnCluster.start() + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + + fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) + } + + override def afterAll() { + yarnCluster.stop() + System.clearProperty("SPARK_YARN_MODE") + super.afterAll() + } + + protected def runSpark( + clientMode: Boolean, + klass: String, + appArgs: Seq[String] = Nil, + sparkArgs: Seq[String] = Nil, + extraClassPath: Seq[String] = Nil, + extraJars: Seq[String] = Nil, + extraConf: Map[String, String] = Map()): Unit = { + val master = if (clientMode) "yarn-client" else "yarn-cluster" + val props = new Properties() + + props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) + + val childClasspath = logConfDir.getAbsolutePath() + + File.pathSeparator + + sys.props("java.class.path") + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator) + props.setProperty("spark.driver.extraClassPath", childClasspath) + props.setProperty("spark.executor.extraClassPath", childClasspath) + + // SPARK-4267: make sure java options are propagated correctly. + props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") + props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") + + yarnCluster.getConfig().foreach { e => + props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) + } + + sys.props.foreach { case (k, v) => + if (k.startsWith("spark.")) { + props.setProperty(k, v) + } + } + + extraConf.foreach { case (k, v) => props.setProperty(k, v) } + + val propsFile = File.createTempFile("spark", ".properties", tempDir) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + props.store(writer, "Spark properties.") + writer.close() + + val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil + val mainArgs = + if (klass.endsWith(".py")) { + Seq(klass) + } else { + Seq("--class", klass, fakeSparkJar.getAbsolutePath()) + } + val argv = + Seq( + new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), + "--master", master, + "--num-executors", "1", + "--properties-file", propsFile.getAbsolutePath()) ++ + extraJarArgs ++ + sparkArgs ++ + mainArgs ++ + appArgs + + Utils.executeAndGetOutput(argv, + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) + } + + /** + * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide + * any sort of error when the job process finishes successfully, but the job itself fails. So + * the tests enforce that something is written to a file after everything is ok to indicate + * that the job succeeded. + */ + protected def checkResult(result: File): Unit = { + checkResult(result, "success") + } + + protected def checkResult(result: File, expected: String): Unit = { + val resultString = Files.toString(result, UTF_8) + resultString should be (expected) + } + + protected def mainClassName(klass: Class[_]): String = { + klass.getName().stripSuffix("$") + } + +} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala index 63cae5370bc1b..4a681a12badbb 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala @@ -39,110 +39,16 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, import org.apache.spark.util.Utils /** -* Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN -* applications, and require the Spark assembly to be built before they can be successfully -* run. -*/ -class ExternalShuffleSuite extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { - - // log4j configuration for the YARN containers, so that their output is collected - // by YARN instead of trying to overwrite unit-tests.log. - private val LOG4J_CONF = """ - |log4j.rootCategory=DEBUG, console - |log4j.appender.console=org.apache.log4j.ConsoleAppender - |log4j.appender.console.target=System.err - |log4j.appender.console.layout=org.apache.log4j.PatternLayout - |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - """.stripMargin - - private val TEST_PYFILE = """ - |import mod1, mod2 - |import sys - |from operator import add - | - |from pyspark import SparkConf , SparkContext - |if __name__ == "__main__": - | if len(sys.argv) != 2: - | print >> sys.stderr, "Usage: test.py [result file]" - | exit(-1) - | sc = SparkContext(conf=SparkConf()) - | status = open(sys.argv[1],'w') - | result = "failure" - | rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func()) - | cnt = rdd.count() - | if cnt == 10: - | result = "success" - | status.write(result) - | status.close() - | sc.stop() - """.stripMargin - - private val TEST_PYMODULE = """ - |def func(): - | return 42 - """.stripMargin - - private var yarnCluster: MiniYARNCluster = _ - private var tempDir: File = _ - private var fakeSparkJar: File = _ - private var hadoopConfDir: File = _ - private var logConfDir: File = _ + * Integration test for the external shuffle service with a yarn mini-cluster + */ +class ExternalShuffleSuite extends BaseYarnClusterSuite { - override def beforeAll() { - super.beforeAll() - - tempDir = Utils.createTempDir() - logConfDir = new File(tempDir, "log4j") - logConfDir.mkdir() - System.setProperty("SPARK_YARN_MODE", "true") - - val logConfFile = new File(logConfDir, "log4j.properties") - Files.write(LOG4J_CONF, logConfFile, UTF_8) - - yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + override def yarnConfig: YarnConfiguration = { val yarnConfig = new YarnConfiguration() yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), "org.apache.spark.network.yarn.YarnShuffleService") - yarnCluster.init(yarnConfig) - yarnCluster.start() - - // There's a race in MiniYARNCluster in which start() may return before the RM has updated - // its address in the configuration. You can see this in the logs by noticing that when - // MiniYARNCluster prints the address, it still has port "0" assigned, although later the - // test works sometimes: - // - // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 - // - // That log message prints the contents of the RM_ADDRESS config variable. If you check it - // later on, it looks something like this: - // - // INFO YarnClusterSuite: RM address in configuration is blah:42631 - // - // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't - // done so in a timely manner (defined to be 10 seconds). - val config = yarnCluster.getConfig() - val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) - while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { - if (System.currentTimeMillis() > deadline) { - throw new IllegalStateException("Timed out waiting for RM to come up.") - } - logDebug("RM address still not set in configuration, waiting...") - TimeUnit.MILLISECONDS.sleep(100) - } - - logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") - - fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) - hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) - assert(hadoopConfDir.mkdir()) - File.createTempFile("token", ".txt", hadoopConfDir) - } - - override def afterAll() { - yarnCluster.stop() - System.clearProperty("SPARK_YARN_MODE") - super.afterAll() + yarnConfig } test("external shuffle service") { @@ -151,95 +57,10 @@ class ExternalShuffleSuite extends SparkFunSuite with BeforeAndAfterAll with Mat false, mainClassName(ExternalShuffleDriver.getClass), appArgs = Seq(result.getAbsolutePath()), -// extraConf = Map() extraConf = Map("spark.shuffle.service.enabled" -> "true") ) checkResult(result) } - - private def runSpark( - clientMode: Boolean, - klass: String, - appArgs: Seq[String] = Nil, - sparkArgs: Seq[String] = Nil, - extraClassPath: Seq[String] = Nil, - extraJars: Seq[String] = Nil, - extraConf: Map[String, String] = Map()): Unit = { - val master = if (clientMode) "yarn-client" else "yarn-cluster" - val props = new Properties() - - props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) - - val childClasspath = logConfDir.getAbsolutePath() + - File.pathSeparator + - sys.props("java.class.path") + - File.pathSeparator + - extraClassPath.mkString(File.pathSeparator) - props.setProperty("spark.driver.extraClassPath", childClasspath) - props.setProperty("spark.executor.extraClassPath", childClasspath) - - // SPARK-4267: make sure java options are propagated correctly. - props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") - props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") - - yarnCluster.getConfig().foreach { e => - props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) - } - - sys.props.foreach { case (k, v) => - if (k.startsWith("spark.")) { - props.setProperty(k, v) - } - } - - extraConf.foreach { case (k, v) => props.setProperty(k, v) } - - val propsFile = File.createTempFile("spark", ".properties", tempDir) - val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) - props.store(writer, "Spark properties.") - writer.close() - - val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil - val mainArgs = - if (klass.endsWith(".py")) { - Seq(klass) - } else { - Seq("--class", klass, fakeSparkJar.getAbsolutePath()) - } - val argv = - Seq( - new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), - "--master", master, - "--num-executors", "1", - "--properties-file", propsFile.getAbsolutePath()) ++ - extraJarArgs ++ - sparkArgs ++ - mainArgs ++ - appArgs - - Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) - } - - /** - * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide - * any sort of error when the job process finishes successfully, but the job itself fails. So - * the tests enforce that something is written to a file after everything is ok to indicate - * that the job succeeded. - */ - private def checkResult(result: File): Unit = { - checkResult(result, "success") - } - - private def checkResult(result: File, expected: String): Unit = { - val resultString = Files.toString(result, UTF_8) - resultString should be (expected) - } - - private def mainClassName(klass: Class[_]): String = { - klass.getName().stripSuffix("$") - } - } private object ExternalShuffleDriver extends Logging with Matchers { @@ -260,16 +81,15 @@ private object ExternalShuffleDriver extends Logging with Matchers { } val sc = new SparkContext(new SparkConf() - .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) - .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) + .setAppName("External Shuffle Test")) val conf = sc.getConf val status = new File(args(0)) var result = "failure" try { - val data = sc.parallelize(1 to 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. + val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. collect().toSet sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - data should be (Set(1, 2, 3, 4)) + data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet) result = "success" } finally { sc.stop() diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 15021eccafc93..9ba3fc419bcba 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -43,17 +43,9 @@ import org.apache.spark.util.Utils * applications, and require the Spark assembly to be built before they can be successfully * run. */ -class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { - - // log4j configuration for the YARN containers, so that their output is collected - // by YARN instead of trying to overwrite unit-tests.log. - private val LOG4J_CONF = """ - |log4j.rootCategory=DEBUG, console - |log4j.appender.console=org.apache.log4j.ConsoleAppender - |log4j.appender.console.target=System.err - |log4j.appender.console.layout=org.apache.log4j.PatternLayout - |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - """.stripMargin +class YarnClusterSuite extends BaseYarnClusterSuite { + + override def yarnConfig: YarnConfiguration = new YarnConfiguration() private val TEST_PYFILE = """ |import mod1, mod2 @@ -82,65 +74,6 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher | return 42 """.stripMargin - private var yarnCluster: MiniYARNCluster = _ - private var tempDir: File = _ - private var fakeSparkJar: File = _ - private var hadoopConfDir: File = _ - private var logConfDir: File = _ - - override def beforeAll() { - super.beforeAll() - - tempDir = Utils.createTempDir() - logConfDir = new File(tempDir, "log4j") - logConfDir.mkdir() - System.setProperty("SPARK_YARN_MODE", "true") - - val logConfFile = new File(logConfDir, "log4j.properties") - Files.write(LOG4J_CONF, logConfFile, UTF_8) - - yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) - yarnCluster.init(new YarnConfiguration()) - yarnCluster.start() - - // There's a race in MiniYARNCluster in which start() may return before the RM has updated - // its address in the configuration. You can see this in the logs by noticing that when - // MiniYARNCluster prints the address, it still has port "0" assigned, although later the - // test works sometimes: - // - // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 - // - // That log message prints the contents of the RM_ADDRESS config variable. If you check it - // later on, it looks something like this: - // - // INFO YarnClusterSuite: RM address in configuration is blah:42631 - // - // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't - // done so in a timely manner (defined to be 10 seconds). - val config = yarnCluster.getConfig() - val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) - while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { - if (System.currentTimeMillis() > deadline) { - throw new IllegalStateException("Timed out waiting for RM to come up.") - } - logDebug("RM address still not set in configuration, waiting...") - TimeUnit.MILLISECONDS.sleep(100) - } - - logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") - - fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) - hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) - assert(hadoopConfDir.mkdir()) - File.createTempFile("token", ".txt", hadoopConfDir) - } - - override def afterAll() { - yarnCluster.stop() - System.clearProperty("SPARK_YARN_MODE") - super.afterAll() - } - test("run Spark in yarn-client mode") { testBasicYarnApp(true) } @@ -224,89 +157,6 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher checkResult(executorResult, "OVERRIDDEN") } - private def runSpark( - clientMode: Boolean, - klass: String, - appArgs: Seq[String] = Nil, - sparkArgs: Seq[String] = Nil, - extraClassPath: Seq[String] = Nil, - extraJars: Seq[String] = Nil, - extraConf: Map[String, String] = Map()): Unit = { - val master = if (clientMode) "yarn-client" else "yarn-cluster" - val props = new Properties() - - props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) - - val childClasspath = logConfDir.getAbsolutePath() + - File.pathSeparator + - sys.props("java.class.path") + - File.pathSeparator + - extraClassPath.mkString(File.pathSeparator) - props.setProperty("spark.driver.extraClassPath", childClasspath) - props.setProperty("spark.executor.extraClassPath", childClasspath) - - // SPARK-4267: make sure java options are propagated correctly. - props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") - props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") - - yarnCluster.getConfig().foreach { e => - props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) - } - - sys.props.foreach { case (k, v) => - if (k.startsWith("spark.")) { - props.setProperty(k, v) - } - } - - extraConf.foreach { case (k, v) => props.setProperty(k, v) } - - val propsFile = File.createTempFile("spark", ".properties", tempDir) - val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) - props.store(writer, "Spark properties.") - writer.close() - - val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil - val mainArgs = - if (klass.endsWith(".py")) { - Seq(klass) - } else { - Seq("--class", klass, fakeSparkJar.getAbsolutePath()) - } - val argv = - Seq( - new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), - "--master", master, - "--num-executors", "1", - "--properties-file", propsFile.getAbsolutePath()) ++ - extraJarArgs ++ - sparkArgs ++ - mainArgs ++ - appArgs - - Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) - } - - /** - * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide - * any sort of error when the job process finishes successfully, but the job itself fails. So - * the tests enforce that something is written to a file after everything is ok to indicate - * that the job succeeded. - */ - private def checkResult(result: File): Unit = { - checkResult(result, "success") - } - - private def checkResult(result: File, expected: String): Unit = { - val resultString = Files.toString(result, UTF_8) - resultString should be (expected) - } - - private def mainClassName(klass: Class[_]): String = { - klass.getName().stripSuffix("$") - } - } private[spark] class SaveExecutorInfo extends SparkListener { From 0e9d69b9fc4bdebd5d226778c336f3157f3b065f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 16:04:48 -0500 Subject: [PATCH 10/43] better names --- ...nalShuffleSuite.scala => YarnExternalShuffleSuite.scala} | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) rename yarn/src/test/scala/org/apache/spark/deploy/yarn/{ExternalShuffleSuite.scala => YarnExternalShuffleSuite.scala} (94%) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala similarity index 94% rename from yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala index 4a681a12badbb..43195266cce15 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils /** * Integration test for the external shuffle service with a yarn mini-cluster */ -class ExternalShuffleSuite extends BaseYarnClusterSuite { +class YarnExternalShuffleSuite extends BaseYarnClusterSuite { override def yarnConfig: YarnConfiguration = { val yarnConfig = new YarnConfiguration() @@ -55,7 +55,7 @@ class ExternalShuffleSuite extends BaseYarnClusterSuite { val result = File.createTempFile("result", null, tempDir) runSpark( false, - mainClassName(ExternalShuffleDriver.getClass), + mainClassName(YarnExternalShuffleDriver.getClass), appArgs = Seq(result.getAbsolutePath()), extraConf = Map("spark.shuffle.service.enabled" -> "true") ) @@ -63,7 +63,7 @@ class ExternalShuffleSuite extends BaseYarnClusterSuite { } } -private object ExternalShuffleDriver extends Logging with Matchers { +private object YarnExternalShuffleDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 From d596969df600adcc291f56ab9c105ffcf3218253 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 16:28:00 -0500 Subject: [PATCH 11/43] cleanup imports --- .../spark/deploy/yarn/BaseYarnClusterSuite.scala | 6 ------ .../spark/deploy/yarn/YarnClusterSuite.scala | 15 +++++---------- .../deploy/yarn/YarnExternalShuffleSuite.scala | 16 ++-------------- 3 files changed, 7 insertions(+), 30 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index d0161f8efe3ba..128e996b71fe5 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -18,24 +18,18 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, OutputStreamWriter} -import java.net.URL import java.util.Properties import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ -import scala.collection.mutable import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.ByteStreams import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster import org.scalatest.{BeforeAndAfterAll, Matchers} import org.apache.spark._ -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} import org.apache.spark.util.Utils abstract class BaseYarnClusterSuite diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 9ba3fc419bcba..98bf4db285e92 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -17,25 +17,20 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.io.File import java.net.URL -import java.util.Properties -import java.util.concurrent.TimeUnit -import scala.collection.JavaConversions._ import scala.collection.mutable +import scala.collection.JavaConversions._ import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.ByteStreams -import com.google.common.io.Files +import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.Matchers import org.apache.spark._ +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala index 43195266cce15..62166db8d80c7 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala @@ -17,26 +17,14 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileOutputStream, OutputStreamWriter} -import java.net.URL -import java.util.Properties -import java.util.concurrent.TimeUnit - -import scala.collection.JavaConversions._ -import scala.collection.mutable +import java.io.File import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.ByteStreams import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} -import org.apache.spark.util.Utils /** * Integration test for the external shuffle service with a yarn mini-cluster From dd93dc033429cdbeae72a02bffb0d45ac7401dee Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 31 Jul 2015 23:37:57 -0500 Subject: [PATCH 12/43] test for shuffle service w/ NM restarts --- .../shuffle/ExternalShuffleBlockHandler.java | 3 +- .../shuffle/ExternalShuffleBlockResolver.java | 11 ++- .../network/yarn/YarnShuffleService.java | 7 +- .../spark/network/shuffle/TestUtil.scala | 47 ++++++++++ .../yarn/YarnShuffleServiceSuite.scala | 90 +++++++++++++++++++ 5 files changed, 151 insertions(+), 7 deletions(-) create mode 100644 yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala create mode 100644 yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index a4f1b27a658c1..c2aabb52142a1 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -48,7 +48,8 @@ public class ExternalShuffleBlockHandler extends RpcHandler { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); - private final ExternalShuffleBlockResolver blockManager; + @VisibleForTesting + final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 8a6a790905b22..bd839ed3ca927 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -50,14 +50,16 @@ public class ExternalShuffleBlockResolver { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); // Map containing all registered executors' metadata. - private final ConcurrentMap executors; + @VisibleForTesting + final ConcurrentMap executors; // Single-threaded Java executor used to perform expensive recursive directory deletion. private final Executor directoryCleaner; private final TransportConf conf; - private final File registeredExecutorFile; + @VisibleForTesting + final File registeredExecutorFile; public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) throws IOException, ClassNotFoundException { @@ -233,11 +235,12 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) } /** Simply encodes an executor's full ID, which is appId + execId. */ - private static class AppExecId implements Serializable { + @VisibleForTesting + static class AppExecId implements Serializable { final String appId; final String execId; - private AppExecId(String appId, String execId) { + AppExecId(String appId, String execId) { this.appId = appId; this.execId = execId; } diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 5efdb63c2af1d..59628cd22fef9 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.util.List; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -79,9 +80,11 @@ public class YarnShuffleService extends AuxiliaryService { private TransportServer shuffleServer = null; // Handles registering executors and opening shuffle blocks - private ExternalShuffleBlockHandler blockHandler; + @VisibleForTesting + ExternalShuffleBlockHandler blockHandler; - private File registeredExecutorFile; + @VisibleForTesting + File registeredExecutorFile; public YarnShuffleService() { super("spark_shuffle"); diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala new file mode 100644 index 0000000000000..628f683a9f7c5 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.network.shuffle + +import java.io.File + +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo + +/** + * just a cheat to get package-visible members in tests + */ +object TestUtil { + + def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = { + handler.blockManager + } + + def getExecutorInfo( + appId: String, + execId: String, + resolver: ExternalShuffleBlockResolver + ): Option[ExecutorShuffleInfo] = { + val id = new AppExecId(appId, execId) + Option(resolver.executors.get(id)) + } + + def registeredExecutorFile(resolver: ExternalShuffleBlockResolver): File = { + resolver.registeredExecutorFile + } + + +} diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala new file mode 100644 index 0000000000000..547fdd27f3fd9 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.network.yarn + +import java.io.File + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.yarn.api.records.ApplicationId +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext +import org.scalatest.Matchers + +import org.apache.spark.SparkFunSuite +import org.apache.spark.network.shuffle.TestUtil +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo + +class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { + private[yarn] var yarnConfig: YarnConfiguration = new YarnConfiguration + + { + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle"); + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), + "org.apache.spark.network.yarn.YarnShuffleService"); + + yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => + println("making dir " + dir) + val d = new File(dir) + if (d.exists()) { + FileUtils.deleteDirectory(d) + } + FileUtils.forceMkdir(d) + } + } + + test("executor state kept across NM restart") { + val service: YarnShuffleService = new YarnShuffleService + service.init(yarnConfig) + val appId = ApplicationId.newInstance(0, 0) + val appData: ApplicationInitializationContext = + new ApplicationInitializationContext("user", appId, null) + service.initializeApplication(appData) + + val execStateFile = service.registeredExecutorFile + execStateFile should not be (null) + execStateFile.exists() should be (false) + + val blockHandler = service.blockHandler + val blockResolver = TestUtil.getBlockResolver(blockHandler) + TestUtil.registeredExecutorFile(blockResolver) should be (execStateFile) + + val shuffleInfo = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) + val executor = TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) + executor should be (Some(shuffleInfo)) + + execStateFile.exists() should be (true) + + // now we pretend the shuffle service goes down, and comes back up + service.stop() + + val s2: YarnShuffleService = new YarnShuffleService + s2.init(yarnConfig) + service.registeredExecutorFile should be (execStateFile) + + val handler2 = service.blockHandler + val resolver2 = TestUtil.getBlockResolver(handler2) + + // until we initial the application, don't know about any executors + +// TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) should be (None) + + s2.initializeApplication(appData) + val ex2 = TestUtil.getExecutorInfo(appId.toString, "exec-1", resolver2) + ex2 should be (Some(shuffleInfo)) + } +} From efb61958fd037952f987cfb08062c331946800b6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 00:13:04 -0500 Subject: [PATCH 13/43] proper unit test, and no longer leak if apps stop during NM restart --- .../shuffle/ExternalShuffleBlockHandler.java | 13 +++-- .../shuffle/ExternalShuffleBlockResolver.java | 25 ++-------- .../network/yarn/YarnShuffleService.java | 47 ++++++++++++++++++- .../yarn/YarnShuffleServiceSuite.scala | 32 ++++++++----- 4 files changed, 76 insertions(+), 41 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index c2aabb52142a1..2794a459f8d2d 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; -import org.apache.spark.network.util.TransportConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,10 +32,10 @@ import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; -import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; +import org.apache.spark.network.shuffle.protocol.*; +import org.apache.spark.network.util.TransportConf; + /** * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. @@ -104,4 +103,8 @@ public StreamManager getStreamManager() { public void applicationRemoved(String appId, boolean cleanupLocalDirs) { blockManager.applicationRemoved(appId, cleanupLocalDirs); } + + public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { + blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index bd839ed3ca927..07026c9061d77 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -78,7 +78,6 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF this.registeredExecutorFile = registeredExecutorFile; this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; - reloadRegisteredExecutors(); } /** Registers a new Executor with all the configuration we need to find its shuffle files. */ @@ -235,12 +234,11 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) } /** Simply encodes an executor's full ID, which is appId + execId. */ - @VisibleForTesting - static class AppExecId implements Serializable { - final String appId; + public static class AppExecId implements Serializable { + public final String appId; final String execId; - AppExecId(String appId, String execId) { + public AppExecId(String appId, String execId) { this.appId = appId; this.execId = execId; } @@ -268,23 +266,6 @@ public String toString() { } } - private void reloadRegisteredExecutors() throws IOException, ClassNotFoundException { - if (registeredExecutorFile != null && registeredExecutorFile.exists()) { - ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); - int nExecutors = in.readInt(); - logger.info("Reloading executors from {}", registeredExecutorFile); - for (int i = 0; i < nExecutors; i++) { - AppExecId appExecId = (AppExecId) in.readObject(); - ExecutorShuffleInfo shuffleInfo = (ExecutorShuffleInfo) in.readObject(); - logger.info("Reregistering executor {} with {}", appExecId, shuffleInfo); - executors.put(appExecId, shuffleInfo); - } - in.close(); - } else { - logger.info("No executor info to reload"); - } - } - private void saveRegisteredExecutors() throws IOException { if (registeredExecutorFile != null) { logger.info("Saving registered executors to {}", registeredExecutorFile); diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 59628cd22fef9..1e8b89f11d722 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -18,9 +18,12 @@ package org.apache.spark.network.yarn; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.ObjectInputStream; import java.nio.ByteBuffer; -import java.util.List; +import java.util.*; +import java.util.Map.Entry; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -31,6 +34,8 @@ import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,6 +91,9 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting File registeredExecutorFile; + private Map>> recoveredExecutorRegistrations = + new HashMap>>(); + public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); @@ -108,6 +116,11 @@ protected void serviceInit(Configuration conf) { registeredExecutorFile = findRegisteredExecutorFile(conf.get("yarn.nodemanager.local-dirs").split(",")); + try { + reloadRegisteredExecutors(); + } catch (Exception e) { + logger.error("Failed to load previously registered executors", e); + } TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); // If authentication is enabled, set up the shuffle server to use a @@ -147,6 +160,14 @@ public void initializeApplication(ApplicationInitializationContext context) { } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } + List> executorsForApp = + recoveredExecutorRegistrations.get(appId); + if (executorsForApp != null) { + for (Entry entry: executorsForApp) { + logger.info("re-registering {} with {}", entry.getKey(), entry.getValue()); + blockHandler.reregisterExecutor(entry.getKey(), entry.getValue()); + } + } } @Override @@ -205,4 +226,28 @@ public ByteBuffer getMetaData() { return ByteBuffer.allocate(0); } + private void reloadRegisteredExecutors() throws IOException, ClassNotFoundException { + if (registeredExecutorFile != null && registeredExecutorFile.exists()) { + ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); + int nExecutors = in.readInt(); + logger.info("Reloading executors from {}", registeredExecutorFile); + for (int i = 0; i < nExecutors; i++) { + AppExecId appExecId = (AppExecId) in.readObject(); + ExecutorShuffleInfo shuffleInfo = (ExecutorShuffleInfo) in.readObject(); + logger.info("Recovered executor {} with {}", appExecId, shuffleInfo); + List> executorsForApp = + recoveredExecutorRegistrations.get(appExecId.appId); + if (executorsForApp == null) { + executorsForApp = new ArrayList<>(); + recoveredExecutorRegistrations.put(appExecId.appId, executorsForApp); + } + executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(appExecId, shuffleInfo)); + } + in.close(); + } else { + logger.info("No executor info to reload"); + } + } + + } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 547fdd27f3fd9..a321ffab3e72a 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -57,34 +57,40 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val execStateFile = service.registeredExecutorFile execStateFile should not be (null) execStateFile.exists() should be (false) + val shuffleInfo = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - val blockHandler = service.blockHandler - val blockResolver = TestUtil.getBlockResolver(blockHandler) - TestUtil.registeredExecutorFile(blockResolver) should be (execStateFile) - val shuffleInfo = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) - val executor = TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) - executor should be (Some(shuffleInfo)) + { + val blockHandler = service.blockHandler + val blockResolver = TestUtil.getBlockResolver(blockHandler) + TestUtil.registeredExecutorFile(blockResolver) should be (execStateFile) - execStateFile.exists() should be (true) + blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) + val executor = TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) + executor should be (Some(shuffleInfo)) - // now we pretend the shuffle service goes down, and comes back up - service.stop() + execStateFile.exists() should be (true) + + // now we pretend the shuffle service goes down, and comes back up + service.stop() + } val s2: YarnShuffleService = new YarnShuffleService s2.init(yarnConfig) service.registeredExecutorFile should be (execStateFile) - val handler2 = service.blockHandler + val handler2 = s2.blockHandler val resolver2 = TestUtil.getBlockResolver(handler2) - // until we initial the application, don't know about any executors + // until we initialize the application, don't know about any executors + // that is so that if the application gets removed while the NM was down, it still eventually + // gets purged from our list of apps. -// TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) should be (None) + TestUtil.getExecutorInfo(appId.toString, "exec-1", resolver2) should be (None) s2.initializeApplication(appData) val ex2 = TestUtil.getExecutorInfo(appId.toString, "exec-1", resolver2) ex2 should be (Some(shuffleInfo)) } + } From a36729c7b7dbb9120bc7c57d08c48766c30ddc65 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 00:13:44 -0500 Subject: [PATCH 14/43] cleanup --- .../scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 6497babc7262d..78e27fb7f3337 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -105,7 +105,6 @@ class ExecutorRunnable( // Authentication is not enabled, so just provide dummy metadata ByteBuffer.allocate(0) } - //throw an exception here, and things just hang ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) } From 7504de8ffde710d4f81f7c53a15d11c2874c1a99 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 02:01:21 -0500 Subject: [PATCH 15/43] style --- .../org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index a321ffab3e72a..8ed297da6cea9 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -37,7 +37,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { "org.apache.spark.network.yarn.YarnShuffleService"); yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => - println("making dir " + dir) val d = new File(dir) if (d.exists()) { FileUtils.deleteDirectory(d) From 23994ff0fd84162b150b2cb379eea0f8c1d54e5d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 02:02:01 -0500 Subject: [PATCH 16/43] style --- .../test/scala/org/apache/spark/network/shuffle/TestUtil.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala index 628f683a9f7c5..fc07fa3e30ea4 100644 --- a/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala @@ -29,7 +29,7 @@ object TestUtil { def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = { handler.blockManager } - + def getExecutorInfo( appId: String, execId: String, @@ -43,5 +43,4 @@ object TestUtil { resolver.registeredExecutorFile } - } From 86e0cb9aeebaa3ba25cb35ac1f5e78541b6a2af2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 09:15:13 -0500 Subject: [PATCH 17/43] for tests, shuffle service finds an open port --- .../apache/spark/storage/BlockManager.scala | 12 +++++++-- .../network/yarn/YarnShuffleService.java | 9 ++++++- .../yarn/YarnExternalShuffleSuite.scala | 9 ++++++- ...stUtil.scala => ShuffleTestAccessor.scala} | 2 +- .../yarn/YarnShuffleServiceSuite.scala | 14 +++++----- .../spark/network/yarn/YarnTestAccessor.scala | 26 +++++++++++++++++++ 6 files changed, 60 insertions(+), 12 deletions(-) rename yarn/src/test/scala/org/apache/spark/network/shuffle/{TestUtil.scala => ShuffleTestAccessor.scala} (98%) create mode 100644 yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 86493673d958d..17981163d00c1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -93,8 +93,15 @@ private[spark] class BlockManager( // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. - private val externalShuffleServicePort = - Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + private val externalShuffleServicePort = { + val t = Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + if (t == 0) { + // this is just for testing, when the yarn conf is set to 0 to find an open port + conf.get("spark.shuffle.service.port").toInt + } else { + t + } + } // Check that we're not using external shuffle service with consolidated shuffle files. if (externalShuffleServiceEnabled @@ -191,6 +198,7 @@ private[spark] class BlockManager( executorId, blockTransferService.hostName, blockTransferService.port) shuffleServerId = if (externalShuffleServiceEnabled) { + logInfo(s"external shuffle service port = $externalShuffleServicePort") BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) } else { blockManagerId diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 1e8b89f11d722..603509324083c 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -91,8 +91,12 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting File registeredExecutorFile; + // just for testing when you want to find an open port + @VisibleForTesting + static int boundPort = -1; + private Map>> recoveredExecutorRegistrations = - new HashMap>>(); + new HashMap<>(); public YarnShuffleService() { super("spark_shuffle"); @@ -142,6 +146,9 @@ protected void serviceInit(Configuration conf) { SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); TransportContext transportContext = new TransportContext(transportConf, blockHandler); shuffleServer = transportContext.createServer(port, bootstraps); + // the port should normally be fixed, but for tests its useful to find an open port + port = shuffleServer.getPort(); + boundPort = port; String authEnabledString = authEnabled ? "enabled" : "not enabled"; logger.info("Started YARN shuffle service for Spark on port {}. " + "Authentication is {}. Registered executor file is {}", port, authEnabledString, diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala index 62166db8d80c7..85b5e012048c2 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.apache.spark._ +import org.apache.spark.network.yarn.YarnTestAccessor /** * Integration test for the external shuffle service with a yarn mini-cluster @@ -36,16 +37,22 @@ class YarnExternalShuffleSuite extends BaseYarnClusterSuite { yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), "org.apache.spark.network.yarn.YarnShuffleService") + yarnConfig.set("spark.shuffle.service.port", "0") yarnConfig } test("external shuffle service") { + val shuffleServicePort = YarnTestAccessor.getShuffleServicePort + logInfo("Shuffle service port = " + shuffleServicePort) val result = File.createTempFile("result", null, tempDir) runSpark( false, mainClassName(YarnExternalShuffleDriver.getClass), appArgs = Seq(result.getAbsolutePath()), - extraConf = Map("spark.shuffle.service.enabled" -> "true") + extraConf = Map( + "spark.shuffle.service.enabled" -> "true", + "spark.shuffle.service.port" -> shuffleServicePort.toString + ) ) checkResult(result) } diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala similarity index 98% rename from yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala rename to yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala index fc07fa3e30ea4..cbec03a457866 100644 --- a/yarn/src/test/scala/org/apache/spark/network/shuffle/TestUtil.scala +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala @@ -24,7 +24,7 @@ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo /** * just a cheat to get package-visible members in tests */ -object TestUtil { +object ShuffleTestAccessor { def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = { handler.blockManager diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 8ed297da6cea9..5953932e801ca 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext import org.scalatest.Matchers import org.apache.spark.SparkFunSuite -import org.apache.spark.network.shuffle.TestUtil +import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { @@ -61,11 +61,11 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { { val blockHandler = service.blockHandler - val blockResolver = TestUtil.getBlockResolver(blockHandler) - TestUtil.registeredExecutorFile(blockResolver) should be (execStateFile) + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) - val executor = TestUtil.getExecutorInfo(appId.toString, "exec-1", blockResolver) + val executor = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", blockResolver) executor should be (Some(shuffleInfo)) execStateFile.exists() should be (true) @@ -79,16 +79,16 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { service.registeredExecutorFile should be (execStateFile) val handler2 = s2.blockHandler - val resolver2 = TestUtil.getBlockResolver(handler2) + val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) // until we initialize the application, don't know about any executors // that is so that if the application gets removed while the NM was down, it still eventually // gets purged from our list of apps. - TestUtil.getExecutorInfo(appId.toString, "exec-1", resolver2) should be (None) + ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", resolver2) should be (None) s2.initializeApplication(appData) - val ex2 = TestUtil.getExecutorInfo(appId.toString, "exec-1", resolver2) + val ex2 = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", resolver2) ex2 should be (Some(shuffleInfo)) } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala new file mode 100644 index 0000000000000..b69a97e214928 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.network.yarn + +/** + * just a cheat to get package-visible members in tests + */ +object YarnTestAccessor { + def getShuffleServicePort: Int = { + YarnShuffleService.boundPort + } +} From bdc4b32125a53a0f7a9e5b714a49d44af35b3a56 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 09:15:36 -0500 Subject: [PATCH 18/43] rename --- ...rnalShuffleSuite.scala => YarnShuffleIntegrationSuite.scala} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename yarn/src/test/scala/org/apache/spark/deploy/yarn/{YarnExternalShuffleSuite.scala => YarnShuffleIntegrationSuite.scala} (97%) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala similarity index 97% rename from yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 85b5e012048c2..d723764401504 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnExternalShuffleSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.network.yarn.YarnTestAccessor /** * Integration test for the external shuffle service with a yarn mini-cluster */ -class YarnExternalShuffleSuite extends BaseYarnClusterSuite { +class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { override def yarnConfig: YarnConfiguration = { val yarnConfig = new YarnConfiguration() From bb9d1e6f02a1c40c6df435c7380c161bd929116e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 09:17:07 -0500 Subject: [PATCH 19/43] formatting --- .../yarn/YarnShuffleServiceSuite.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 5953932e801ca..33eaeaff979f1 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -59,20 +59,18 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val shuffleInfo = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - { - val blockHandler = service.blockHandler - val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) - ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + val blockHandler = service.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) - blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) - val executor = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", blockResolver) - executor should be (Some(shuffleInfo)) + blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) + val executor = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", blockResolver) + executor should be (Some(shuffleInfo)) - execStateFile.exists() should be (true) + execStateFile.exists() should be (true) - // now we pretend the shuffle service goes down, and comes back up - service.stop() - } + // now we pretend the shuffle service goes down, and comes back up + service.stop() val s2: YarnShuffleService = new YarnShuffleService s2.init(yarnConfig) From 857331a6a755db705e6c24a6d29acf4fe5c2fcc5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 14:49:55 -0500 Subject: [PATCH 20/43] better tests & comments --- .../network/yarn/YarnShuffleService.java | 15 +++- .../network/shuffle/ShuffleTestAccessor.scala | 6 +- .../yarn/YarnShuffleServiceSuite.scala | 69 +++++++++++++++---- 3 files changed, 72 insertions(+), 18 deletions(-) diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 603509324083c..550113ae19cef 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -88,6 +88,7 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting ExternalShuffleBlockHandler blockHandler; + // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting File registeredExecutorFile; @@ -95,7 +96,8 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting static int boundPort = -1; - private Map>> recoveredExecutorRegistrations = + @VisibleForTesting + Map>> recoveredExecutorRegistrations = new HashMap<>(); public YarnShuffleService() { @@ -118,6 +120,15 @@ private boolean isAuthenticationEnabled() { @Override protected void serviceInit(Configuration conf) { + // In case this NM was killed while there were running spark applications, we need to restore + // lost state for the existing executors. We look for an existing file in the NM's local dirs. + // If we don't find one, then we choose a file to use to save the state next time. However, we + // do *not* immediately register all the executors in that file, just in case the application + // was terminated while the NM was restarting. We wait until yarn tells the service about the + // app again via #initializeApplication, so we know its still running. That is important + // for preventing a leak where the app data would stick around *forever*. This does leave + // a small race -- if the NM restarts *again*, after only some of the existing apps have been + // re-registered, their info will be lost. registeredExecutorFile = findRegisteredExecutorFile(conf.get("yarn.nodemanager.local-dirs").split(",")); try { @@ -167,6 +178,8 @@ public void initializeApplication(ApplicationInitializationContext context) { } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } + // See if we already have data for this app from before the restart -- if so, re-register + // those executors List> executorsForApp = recoveredExecutorRegistrations.get(appId); if (executorsForApp != null) { diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala index cbec03a457866..97da440d636bc 100644 --- a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala @@ -18,6 +18,8 @@ package org.apache.spark.network.shuffle import java.io.File +import org.apache.hadoop.yarn.api.records.ApplicationId + import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo @@ -31,11 +33,11 @@ object ShuffleTestAccessor { } def getExecutorInfo( - appId: String, + appId: ApplicationId, execId: String, resolver: ExternalShuffleBlockResolver ): Option[ExecutorShuffleInfo] = { - val id = new AppExecId(appId, execId) + val id = new AppExecId(appId.toString, execId) Option(resolver.executors.get(id)) } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 33eaeaff979f1..4e35adce13d60 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -46,35 +46,43 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { } test("executor state kept across NM restart") { - val service: YarnShuffleService = new YarnShuffleService + val service = new YarnShuffleService service.init(yarnConfig) - val appId = ApplicationId.newInstance(0, 0) - val appData: ApplicationInitializationContext = - new ApplicationInitializationContext("user", appId, null) - service.initializeApplication(appData) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + service.initializeApplication(app1Data) + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + service.initializeApplication(app2Data) val execStateFile = service.registeredExecutorFile execStateFile should not be (null) execStateFile.exists() should be (false) - val shuffleInfo = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") val blockHandler = service.blockHandler val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) - blockResolver.registerExecutor(appId.toString, "exec-1", shuffleInfo) - val executor = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", blockResolver) - executor should be (Some(shuffleInfo)) + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should + be (Some(shuffleInfo1)) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should + be (Some(shuffleInfo2)) execStateFile.exists() should be (true) // now we pretend the shuffle service goes down, and comes back up service.stop() - val s2: YarnShuffleService = new YarnShuffleService + val s2 = new YarnShuffleService s2.init(yarnConfig) - service.registeredExecutorFile should be (execStateFile) + s2.registeredExecutorFile should be (execStateFile) val handler2 = s2.blockHandler val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) @@ -82,12 +90,43 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { // until we initialize the application, don't know about any executors // that is so that if the application gets removed while the NM was down, it still eventually // gets purged from our list of apps. + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (None) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) - ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", resolver2) should be (None) + // now we reinitialize only one of the apps (as if app2 was stopped during the NM restart) + s2.initializeApplication(app1Data) + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1)) + + + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) + + // Act like the NM restarts one more time + + s2.stop() + + val s3 = new YarnShuffleService + s3.init(yarnConfig) + s3.registeredExecutorFile should be (execStateFile) + + // the second app won't even be in our file of saved executor info + // (this is mostly an implementation detail, by itself its not really an important check ...) + s3.recoveredExecutorRegistrations.get(app1Id.toString) should not be (null) + s3.recoveredExecutorRegistrations.get(app2Id.toString) should be (null) + + val handler3 = s3.blockHandler + val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) + + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (None) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) + + // now if we initialize both those apps, we'll have restored executor info for app 1, + // but for app 2, there won't anything to restore + s3.initializeApplication(app1Data) + s3.initializeApplication(app2Data) + + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) - s2.initializeApplication(appData) - val ex2 = ShuffleTestAccessor.getExecutorInfo(appId.toString, "exec-1", resolver2) - ex2 should be (Some(shuffleInfo)) } } From bdbbf0dd190007c776753869189164fb0b24bca3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sat, 1 Aug 2015 15:09:43 -0500 Subject: [PATCH 21/43] comments, remove some unnecessary changes --- .../shuffle/ExternalShuffleBlockHandler.java | 13 +++++++++++-- .../shuffle/ExternalShuffleBlockResolver.java | 5 ++--- .../shuffle/ExternalShuffleBlockResolverSuite.java | 6 +++--- .../shuffle/ExternalShuffleCleanupSuite.java | 8 ++++---- .../shuffle/ExternalShuffleIntegrationSuite.java | 2 +- .../shuffle/ExternalShuffleSecuritySuite.java | 2 +- yarn/src/test/resources/log4j.properties | 2 +- 7 files changed, 23 insertions(+), 15 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 2794a459f8d2d..69b2e3bab5359 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -51,8 +51,7 @@ public class ExternalShuffleBlockHandler extends RpcHandler { final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) - throws IOException, ClassNotFoundException { + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) { this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } @@ -104,6 +103,16 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { blockManager.applicationRemoved(appId, cleanupLocalDirs); } + /** + * Register an (application, executor) with the given shuffle info. + * + * The "re-" is meant to highlight the intended use of this method -- when this service is + * restarted, this is used to restore the state of executors from before the restart. Normal + * registration will happen via a message handled in receive() + * + * @param appExecId + * @param executorInfo + */ public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 07026c9061d77..3b5a6f0ab8091 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -61,8 +61,7 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final File registeredExecutorFile; - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) - throws IOException, ClassNotFoundException { + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); @@ -73,7 +72,7 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF ExternalShuffleBlockResolver( TransportConf conf, File registeredExecutorFile, - Executor directoryCleaner) throws IOException, ClassNotFoundException { + Executor directoryCleaner) { this.conf = conf; this.registeredExecutorFile = registeredExecutorFile; this.executors = Maps.newConcurrentMap(); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 49509a201f1c4..808a554a774fd 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -59,7 +59,7 @@ public static void afterAll() { } @Test - public void testBadRequests() throws IOException, ClassNotFoundException { + public void testBadRequests() { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { @@ -90,7 +90,7 @@ public void testBadRequests() throws IOException, ClassNotFoundException { } @Test - public void testSortShuffleBlocks() throws IOException, ClassNotFoundException { + public void testSortShuffleBlocks() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); @@ -109,7 +109,7 @@ public void testSortShuffleBlocks() throws IOException, ClassNotFoundException { } @Test - public void testHashShuffleBlocks() throws IOException, ClassNotFoundException { + public void testHashShuffleBlocks() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 2986c15ff97d5..2f4f1d0df478b 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -38,7 +38,7 @@ public class ExternalShuffleCleanupSuite { TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); @Test - public void noCleanupAndCleanup() throws IOException, ClassNotFoundException { + public void noCleanupAndCleanup() throws IOException { TestShuffleDataContext dataContext = createSomeData(); ExternalShuffleBlockResolver resolver = @@ -55,7 +55,7 @@ public void noCleanupAndCleanup() throws IOException, ClassNotFoundException { } @Test - public void cleanupUsesExecutor() throws IOException, ClassNotFoundException { + public void cleanupUsesExecutor() throws IOException { TestShuffleDataContext dataContext = createSomeData(); final AtomicBoolean cleanupCalled = new AtomicBoolean(false); @@ -79,7 +79,7 @@ public void cleanupUsesExecutor() throws IOException, ClassNotFoundException { } @Test - public void cleanupMultipleExecutors() throws IOException, ClassNotFoundException { + public void cleanupMultipleExecutors() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); @@ -95,7 +95,7 @@ public void cleanupMultipleExecutors() throws IOException, ClassNotFoundExceptio } @Test - public void cleanupOnlyRemovedApp() throws IOException, ClassNotFoundException { + public void cleanupOnlyRemovedApp() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index c2de8d2111f0e..a3f9a38b1aeb9 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -73,7 +73,7 @@ public class ExternalShuffleIntegrationSuite { }; @BeforeClass - public static void beforeAll() throws IOException, ClassNotFoundException { + public static void beforeAll() throws IOException { Random rand = new Random(); for (byte[] block : exec0Blocks) { diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index d57b65ca429a6..5da708d325cf8 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -43,7 +43,7 @@ public class ExternalShuffleSecuritySuite { TransportServer server; @Before - public void beforeEach() throws IOException, ClassNotFoundException { + public void beforeEach() { TransportContext context = new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index 1eb7ca41859e6..6b8a5dbf6373e 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -25,4 +25,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.apache.hadoop=INFO +log4j.logger.org.apache.hadoop=WARN From 62586a68e3ec3e22aad58d7b74599371eb352b30 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 14:16:47 -0500 Subject: [PATCH 22/43] just serialize the whole executors map --- .../shuffle/ExternalShuffleBlockResolver.java | 21 ++++++++++--------- .../network/yarn/YarnShuffleService.java | 18 +++++++--------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 3b5a6f0ab8091..2c87e6d3bb01a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -265,19 +265,20 @@ public String toString() { } } + /** + * write out the set of registered executors to a file so we can reload them on restart. + * You must have a lock on executors when calling this + */ private void saveRegisteredExecutors() throws IOException { if (registeredExecutorFile != null) { - logger.info("Saving registered executors to {}", registeredExecutorFile); - ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(registeredExecutorFile)); - // synchronize so we can write out the size :( - synchronized (executors) { - out.writeInt(executors.size()); - for (Map.Entry e: executors.entrySet()) { - out.writeObject(e.getKey()); - out.writeObject(e.getValue()); - } - } + File tmp = File.createTempFile("registeredExecutors",".bin"); + ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(tmp)); + out.writeObject(executors); out.close(); + if (!tmp.renameTo(registeredExecutorFile)) { + logger.error("Failed to move registered executors to {}", registeredExecutorFile); + } + logger.info("Saving registered executors to {}", registeredExecutorFile); } } } diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 550113ae19cef..efb80d1a27dba 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -130,7 +130,7 @@ protected void serviceInit(Configuration conf) { // a small race -- if the NM restarts *again*, after only some of the existing apps have been // re-registered, their info will be lost. registeredExecutorFile = - findRegisteredExecutorFile(conf.get("yarn.nodemanager.local-dirs").split(",")); + findRegisteredExecutorFile(conf.getStrings("yarn.nodemanager.local-dirs")); try { reloadRegisteredExecutors(); } catch (Exception e) { @@ -248,20 +248,18 @@ public ByteBuffer getMetaData() { private void reloadRegisteredExecutors() throws IOException, ClassNotFoundException { if (registeredExecutorFile != null && registeredExecutorFile.exists()) { - ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); - int nExecutors = in.readInt(); logger.info("Reloading executors from {}", registeredExecutorFile); - for (int i = 0; i < nExecutors; i++) { - AppExecId appExecId = (AppExecId) in.readObject(); - ExecutorShuffleInfo shuffleInfo = (ExecutorShuffleInfo) in.readObject(); - logger.info("Recovered executor {} with {}", appExecId, shuffleInfo); + ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); + Map registeredExecutors = + (Map) in.readObject(); + for (Entry e: registeredExecutors.entrySet()) { List> executorsForApp = - recoveredExecutorRegistrations.get(appExecId.appId); + recoveredExecutorRegistrations.get(e.getKey().appId); if (executorsForApp == null) { executorsForApp = new ArrayList<>(); - recoveredExecutorRegistrations.put(appExecId.appId, executorsForApp); + recoveredExecutorRegistrations.put(e.getKey().appId, executorsForApp); } - executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(appExecId, shuffleInfo)); + executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); } in.close(); } else { From 245db19e5ac43dc6717da508f015498e07e09cad Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 14:17:25 -0500 Subject: [PATCH 23/43] style --- .../spark/deploy/yarn/YarnShuffleIntegrationSuite.scala | 4 ++-- .../apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 6 ------ 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index d723764401504..404becf57d81c 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.network.yarn.YarnTestAccessor +import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} /** * Integration test for the external shuffle service with a yarn mini-cluster @@ -36,7 +36,7 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { val yarnConfig = new YarnConfiguration() yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), - "org.apache.spark.network.yarn.YarnShuffleService") + classOf[YarnShuffleService].getCanonicalName) yarnConfig.set("spark.shuffle.service.port", "0") yarnConfig } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 4e35adce13d60..4f4b2b00b8282 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -63,7 +63,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") - val blockHandler = service.blockHandler val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) @@ -79,7 +78,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { // now we pretend the shuffle service goes down, and comes back up service.stop() - val s2 = new YarnShuffleService s2.init(yarnConfig) s2.registeredExecutorFile should be (execStateFile) @@ -97,13 +95,10 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { s2.initializeApplication(app1Data) ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1)) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) // Act like the NM restarts one more time - s2.stop() - val s3 = new YarnShuffleService s3.init(yarnConfig) s3.registeredExecutorFile should be (execStateFile) @@ -126,7 +121,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) - } } From 55f49fc9865100cc45cdeb37c578ddb9ce5849bf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 16:54:02 -0500 Subject: [PATCH 24/43] make sure the service doesnt die if the registered executor file is corrupt; add tests --- .../network/yarn/YarnShuffleService.java | 59 ++++++++----- .../yarn/YarnShuffleIntegrationSuite.scala | 6 ++ .../yarn/YarnShuffleServiceSuite.scala | 86 ++++++++++++++++--- .../spark/network/yarn/YarnTestAccessor.scala | 23 +++++ 4 files changed, 139 insertions(+), 35 deletions(-) diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index efb80d1a27dba..7714ec804a322 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -17,10 +17,7 @@ package org.apache.spark.network.yarn; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.ObjectInputStream; +import java.io.*; import java.nio.ByteBuffer; import java.util.*; import java.util.Map.Entry; @@ -96,13 +93,18 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting static int boundPort = -1; + // just for integration tests that want to look at this file -- in general not sensible as + // a static @VisibleForTesting - Map>> recoveredExecutorRegistrations = - new HashMap<>(); + static YarnShuffleService instance; + + @VisibleForTesting + Map>> recoveredExecutorRegistrations = null; public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); + instance = this; } /** @@ -128,12 +130,13 @@ protected void serviceInit(Configuration conf) { // app again via #initializeApplication, so we know its still running. That is important // for preventing a leak where the app data would stick around *forever*. This does leave // a small race -- if the NM restarts *again*, after only some of the existing apps have been - // re-registered, their info will be lost. + // re-registered, the info of the remaining apps is lost. registeredExecutorFile = findRegisteredExecutorFile(conf.getStrings("yarn.nodemanager.local-dirs")); try { - reloadRegisteredExecutors(); + recoveredExecutorRegistrations = reloadRegisteredExecutors(); } catch (Exception e) { + recoveredExecutorRegistrations = new HashMap<>(); logger.error("Failed to load previously registered executors", e); } @@ -246,25 +249,37 @@ public ByteBuffer getMetaData() { return ByteBuffer.allocate(0); } - private void reloadRegisteredExecutors() throws IOException, ClassNotFoundException { + private Map>> reloadRegisteredExecutors() + throws IOException, ClassNotFoundException { if (registeredExecutorFile != null && registeredExecutorFile.exists()) { logger.info("Reloading executors from {}", registeredExecutorFile); - ObjectInputStream in = new ObjectInputStream(new FileInputStream(registeredExecutorFile)); - Map registeredExecutors = - (Map) in.readObject(); - for (Entry e: registeredExecutors.entrySet()) { - List> executorsForApp = - recoveredExecutorRegistrations.get(e.getKey().appId); - if (executorsForApp == null) { - executorsForApp = new ArrayList<>(); - recoveredExecutorRegistrations.put(e.getKey().appId, executorsForApp); - } - executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); - } - in.close(); + return reloadRegisteredExecutors(registeredExecutorFile); } else { logger.info("No executor info to reload"); + return new HashMap<>(); + } + } + + + @VisibleForTesting + static Map>> reloadRegisteredExecutors( + File file + ) throws IOException, ClassNotFoundException { + Map>> result = new HashMap<>(); + ObjectInputStream in = new ObjectInputStream(new FileInputStream(file)); + Map registeredExecutors = + (Map) in.readObject(); + for (Entry e: registeredExecutors.entrySet()) { + List> executorsForApp = + result.get(e.getKey().appId); + if (executorsForApp == null) { + executorsForApp = new ArrayList<>(); + result.put(e.getKey().appId, executorsForApp); + } + executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); } + in.close(); + return result; } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 404becf57d81c..5175048d1f01e 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -43,6 +43,10 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { test("external shuffle service") { val shuffleServicePort = YarnTestAccessor.getShuffleServicePort + val shuffleService = YarnTestAccessor.getShuffleServiceInstance + + assert(!YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) + logInfo("Shuffle service port = " + shuffleServicePort) val result = File.createTempFile("result", null, tempDir) runSpark( @@ -55,6 +59,8 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { ) ) checkResult(result) + assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) + assert(!YarnTestAccessor.loadSavedExecutors(shuffleService).isEmpty) } } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 4f4b2b00b8282..6f6c534a7ab52 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -16,22 +16,23 @@ */ package org.apache.spark.network.yarn -import java.io.File +import java.io.{PrintWriter, File} import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext -import org.scalatest.Matchers +import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.SparkFunSuite import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo -class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { +class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { private[yarn] var yarnConfig: YarnConfiguration = new YarnConfiguration - { + + override def beforeEach(): Unit = { yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle"); yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), "org.apache.spark.network.yarn.YarnShuffleService"); @@ -45,25 +46,37 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { } } + var s1: YarnShuffleService = null + var s2: YarnShuffleService = null + var s3: YarnShuffleService = null + + override def afterEach(): Unit = { + Seq(s1, s2, s3).foreach { service => + if (service != null) { + service.stop() + } + } + } + test("executor state kept across NM restart") { - val service = new YarnShuffleService - service.init(yarnConfig) + s1 = new YarnShuffleService + s1.init(yarnConfig) val app1Id = ApplicationId.newInstance(0, 1) val app1Data: ApplicationInitializationContext = new ApplicationInitializationContext("user", app1Id, null) - service.initializeApplication(app1Data) + s1.initializeApplication(app1Data) val app2Id = ApplicationId.newInstance(0, 2) val app2Data: ApplicationInitializationContext = new ApplicationInitializationContext("user", app2Id, null) - service.initializeApplication(app2Data) + s1.initializeApplication(app2Data) - val execStateFile = service.registeredExecutorFile + val execStateFile = s1.registeredExecutorFile execStateFile should not be (null) execStateFile.exists() should be (false) val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") - val blockHandler = service.blockHandler + val blockHandler = s1.blockHandler val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) @@ -77,8 +90,8 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { execStateFile.exists() should be (true) // now we pretend the shuffle service goes down, and comes back up - service.stop() - val s2 = new YarnShuffleService + s1.stop() + s2 = new YarnShuffleService s2.init(yarnConfig) s2.registeredExecutorFile should be (execStateFile) @@ -99,7 +112,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { // Act like the NM restarts one more time s2.stop() - val s3 = new YarnShuffleService + s3 = new YarnShuffleService s3.init(yarnConfig) s3.registeredExecutorFile should be (execStateFile) @@ -121,6 +134,53 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) + s3.stop() + } + + + test("shuffle service should be robust to corrupt registered executor file") { + s1 = new YarnShuffleService + s1.init(yarnConfig) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + s1.initializeApplication(app1Data) + + val execStateFile = s1.registeredExecutorFile + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + + val blockHandler = s1.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + + // now we pretend the shuffle service goes down, and comes back up. But we'll also + // make a corrupt registeredExecutor File + s1.stop() + + val out = new PrintWriter(execStateFile) + out.println("42") + out.close() + + s2 = new YarnShuffleService + s2.init(yarnConfig) + s2.registeredExecutorFile should be (execStateFile) + + val handler2 = s2.blockHandler + val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) + + // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ... + s2.initializeApplication(app1Data) + // however, when we initialize a totally new app2, everything is still happy + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + s2.initializeApplication(app2Data) + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") + resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2)) + s2.stop() } } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala index b69a97e214928..ef29b03d35e1c 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -16,6 +16,14 @@ */ package org.apache.spark.network.yarn +import java.io.File +import java.util.{List => JList, Map => JMap} +import java.util.Map.Entry + + +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo + /** * just a cheat to get package-visible members in tests */ @@ -23,4 +31,19 @@ object YarnTestAccessor { def getShuffleServicePort: Int = { YarnShuffleService.boundPort } + + def getShuffleServiceInstance: YarnShuffleService = { + YarnShuffleService.instance + } + + def getRegisteredExecutorFile(service: YarnShuffleService): File = { + service.registeredExecutorFile + } + + def loadSavedExecutors( + service: YarnShuffleService + ): JMap[String, JList[Entry[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo]]] = { + YarnShuffleService.reloadRegisteredExecutors(service.registeredExecutorFile) + } + } From 44928353fd630b46cdc3e97c76f3a7593955539a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 17:17:21 -0500 Subject: [PATCH 25/43] lol, dont use a PrintWriter b/c of scalastyle checks --- .../apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 6f6c534a7ab52..d76b406c5a3fa 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.network.yarn -import java.io.{PrintWriter, File} +import java.io.{DataOutputStream, FileOutputStream, PrintWriter, File} import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId @@ -159,8 +159,8 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd // make a corrupt registeredExecutor File s1.stop() - val out = new PrintWriter(execStateFile) - out.println("42") + val out = new DataOutputStream(new FileOutputStream(execStateFile)) + out.writeInt(42) out.close() s2 = new YarnShuffleService From f729e2b6aac60f380a337e1bf0b42858006ab7b0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 19:53:01 -0500 Subject: [PATCH 26/43] debugging --- .../network/yarn/YarnShuffleServiceSuite.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index d76b406c5a3fa..0f205e346925d 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.network.yarn import java.io.{DataOutputStream, FileOutputStream, PrintWriter, File} +import scala.annotation.tailrec + import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -43,6 +45,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd FileUtils.deleteDirectory(d) } FileUtils.forceMkdir(d) + logInfo(s"creating yarn.nodemanager.local-dirs: $d") } } @@ -87,7 +90,16 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should be (Some(shuffleInfo2)) - execStateFile.exists() should be (true) + if (!execStateFile.exists()) { + @tailrec def findExistingParent(file: File): File = { + if (file == null) file + else if (file.exists()) file + else findExistingParent(file.getParentFile()) + } + val existingParent = findExistingParent(execStateFile) + assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent") + } + assert(execStateFile.exists(), s"$execStateFile did not exist") // now we pretend the shuffle service goes down, and comes back up s1.stop() From d7450f0e3209c776d333090836ffc57de8c3b34c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 3 Aug 2015 20:10:27 -0500 Subject: [PATCH 27/43] style --- .../org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 0f205e346925d..db373869f356c 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -35,7 +35,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd override def beforeEach(): Unit = { - yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle"); + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), "org.apache.spark.network.yarn.YarnShuffleService"); From 59800b77f0597608691af1323880f54c6a269ef4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Aug 2015 15:23:25 -0500 Subject: [PATCH 28/43] Files.move in case renaming is unsupported --- .../spark/network/shuffle/ExternalShuffleBlockResolver.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 2c87e6d3bb01a..ea26fb6f9d038 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -27,6 +27,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.collect.Maps; +import com.google.common.io.Files; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -275,9 +276,7 @@ private void saveRegisteredExecutors() throws IOException { ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(tmp)); out.writeObject(executors); out.close(); - if (!tmp.renameTo(registeredExecutorFile)) { - logger.error("Failed to move registered executors to {}", registeredExecutorFile); - } + Files.move(tmp, registeredExecutorFile); logger.info("Saving registered executors to {}", registeredExecutorFile); } } From d596410ef6d0152d78e3bc1b8054531ba46f70d3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Aug 2015 16:26:48 -0500 Subject: [PATCH 29/43] store executor data in leveldb --- network/shuffle/pom.xml | 6 +++ .../shuffle/ExternalShuffleBlockResolver.java | 25 +++++++-- .../network/yarn/YarnShuffleService.java | 51 +++++++++++++------ .../yarn/YarnShuffleServiceSuite.scala | 16 +++++- 4 files changed, 77 insertions(+), 21 deletions(-) diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index 532463e96fbb7..420d726e63b04 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -43,6 +43,12 @@ ${project.version} + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + + org.slf4j diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index ea26fb6f9d038..e4887485d2633 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -25,9 +25,13 @@ import java.util.concurrent.Executors; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; import com.google.common.base.Objects; import com.google.common.collect.Maps; import com.google.common.io.Files; +import org.fusesource.leveldbjni.JniDBFactory; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -272,11 +276,22 @@ public String toString() { */ private void saveRegisteredExecutors() throws IOException { if (registeredExecutorFile != null) { - File tmp = File.createTempFile("registeredExecutors",".bin"); - ObjectOutputStream out = new ObjectOutputStream(new FileOutputStream(tmp)); - out.writeObject(executors); - out.close(); - Files.move(tmp, registeredExecutorFile); + Options options = new Options(); + options.createIfMissing(true); + JniDBFactory factory = new JniDBFactory(); + DB db = null; + try { + db = factory.open(registeredExecutorFile, options); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bytes); + out.writeObject(executors); + out.close(); + db.put("registeredExecutors".getBytes(Charsets.UTF_8), bytes.toByteArray()); + } finally { + if (db != null) { + db.close(); + } + } logger.info("Saving registered executors to {}", registeredExecutorFile); } } diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 7714ec804a322..4801df391e645 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -23,7 +23,9 @@ import java.util.Map.Entry; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; import com.google.common.collect.Lists; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.AuxiliaryService; @@ -33,6 +35,10 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB.DBException; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -221,12 +227,12 @@ public void stopContainer(ContainerTerminationContext context) { private File findRegisteredExecutorFile(String[] localDirs) { for (String dir: localDirs) { - File f = new File(dir, "registeredExecutors.bin"); + File f = new File(dir, "registeredExecutors.ldb"); if (f.exists()) { return f; } } - return new File(localDirs[0], "registeredExecutors.bin"); + return new File(localDirs[0], "registeredExecutors.ldb"); } /** @@ -263,22 +269,37 @@ private Map>> reloadRegistere @VisibleForTesting static Map>> reloadRegisteredExecutors( - File file - ) throws IOException, ClassNotFoundException { + File file) throws IOException, ClassNotFoundException { Map>> result = new HashMap<>(); - ObjectInputStream in = new ObjectInputStream(new FileInputStream(file)); - Map registeredExecutors = - (Map) in.readObject(); - for (Entry e: registeredExecutors.entrySet()) { - List> executorsForApp = - result.get(e.getKey().appId); - if (executorsForApp == null) { - executorsForApp = new ArrayList<>(); - result.put(e.getKey().appId, executorsForApp); + Options options = new Options(); + options.createIfMissing(true); + JniDBFactory factory = new JniDBFactory(); + DB db = null; + try { + db = factory.open(file, options); + ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream( + db.get("registeredExecutors".getBytes(Charsets.UTF_8)))); + Map registeredExecutors = + (Map) in.readObject(); + for (Entry e: registeredExecutors.entrySet()) { + List> executorsForApp = + result.get(e.getKey().appId); + if (executorsForApp == null) { + executorsForApp = new ArrayList<>(); + result.put(e.getKey().appId, executorsForApp); + } + executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); + } + in.close(); + } catch (DBException dbe) { + // blow the corrupt db away, so we can still write out OK data for future executors + FileUtils.deleteDirectory(file); + throw dbe; + } finally { + if (db != null) { + db.close(); } - executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); } - in.close(); return result; } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index db373869f356c..c39df0fa7f6c2 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -171,7 +171,9 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd // make a corrupt registeredExecutor File s1.stop() - val out = new DataOutputStream(new FileOutputStream(execStateFile)) + execStateFile.listFiles().foreach{_.delete()} + + val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT")) out.writeInt(42) out.close() @@ -193,6 +195,18 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2)) s2.stop() + + // another stop & restart should be fine though (eg., we recover from previous corruption) + s3 = new YarnShuffleService + s3.init(yarnConfig) + s3.registeredExecutorFile should be (execStateFile) + val handler3 = s3.blockHandler + val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) + + s3.initializeApplication(app2Data) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2)) + s3.stop() + } } From 694934cdcf862c4f80c106282b7b8a6272c801f8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Aug 2015 10:34:22 -0500 Subject: [PATCH 30/43] only open leveldb connection once per service --- .../shuffle/ExternalShuffleBlockHandler.java | 4 ++ .../shuffle/ExternalShuffleBlockResolver.java | 51 ++++++++++++------- .../network/yarn/YarnShuffleService.java | 3 ++ .../yarn/YarnShuffleServiceSuite.scala | 3 +- 4 files changed, 40 insertions(+), 21 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 82676d59fa305..e8b25b8f9af98 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -122,4 +122,8 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); } + + public void close() { + blockManager.close(); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index e4887485d2633..db396e0a70f6e 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -28,7 +28,6 @@ import com.google.common.base.Charsets; import com.google.common.base.Objects; import com.google.common.collect.Maps; -import com.google.common.io.Files; import org.fusesource.leveldbjni.JniDBFactory; import org.iq80.leveldb.DB; import org.iq80.leveldb.Options; @@ -65,6 +64,7 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final File registeredExecutorFile; + private final DB db; public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( @@ -80,6 +80,21 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF Executor directoryCleaner) { this.conf = conf; this.registeredExecutorFile = registeredExecutorFile; + if (registeredExecutorFile != null) { + Options options = new Options(); + options.createIfMissing(true); + JniDBFactory factory = new JniDBFactory(); + DB tmpDb; + try { + tmpDb = factory.open(registeredExecutorFile, options); + } catch (IOException e) { + logger.info("Error opening leveldb file {}", registeredExecutorFile, e); + tmpDb = null; + } + db = tmpDb; + } else { + db = null; + } this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; } @@ -237,6 +252,16 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) return new File(new File(localDir, String.format("%02x", subDirId)), filename); } + void close() { + if (db != null) { + try { + db.close(); + } catch (IOException e) { + logger.error("Exception closing leveldb with registered executors", e); + } + } + } + /** Simply encodes an executor's full ID, which is appId + execId. */ public static class AppExecId implements Serializable { public final String appId; @@ -275,24 +300,12 @@ public String toString() { * You must have a lock on executors when calling this */ private void saveRegisteredExecutors() throws IOException { - if (registeredExecutorFile != null) { - Options options = new Options(); - options.createIfMissing(true); - JniDBFactory factory = new JniDBFactory(); - DB db = null; - try { - db = factory.open(registeredExecutorFile, options); - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - out.writeObject(executors); - out.close(); - db.put("registeredExecutors".getBytes(Charsets.UTF_8), bytes.toByteArray()); - } finally { - if (db != null) { - db.close(); - } - } - logger.info("Saving registered executors to {}", registeredExecutorFile); + if (db != null) { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bytes); + out.writeObject(executors); + out.close(); + db.put("registeredExecutors".getBytes(Charsets.UTF_8), bytes.toByteArray()); } } } diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 4801df391e645..1634a89c9980a 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -244,6 +244,9 @@ protected void serviceStop() { if (shuffleServer != null) { shuffleServer.close(); } + if (blockHandler != null) { + blockHandler.close(); + } } catch (Exception e) { logger.error("Exception when stopping service", e); } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index c39df0fa7f6c2..d03b287b853c1 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -37,7 +37,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd override def beforeEach(): Unit = { yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), - "org.apache.spark.network.yarn.YarnShuffleService"); + classOf[YarnShuffleService].getCanonicalName); yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => val d = new File(dir) @@ -75,7 +75,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd val execStateFile = s1.registeredExecutorFile execStateFile should not be (null) - execStateFile.exists() should be (false) val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") From c878fbe047bc3a197dfbb2d793e04b6eb2db53c7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Aug 2015 09:59:27 -0500 Subject: [PATCH 31/43] better explanation of shuffle service port handling --- .../scala/org/apache/spark/storage/BlockManager.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 17981163d00c1..fbb800869c528 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -94,12 +94,14 @@ private[spark] class BlockManager( // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. private val externalShuffleServicePort = { - val t = Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt - if (t == 0) { - // this is just for testing, when the yarn conf is set to 0 to find an open port + val tmpPort = Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + if (tmpPort == 0) { + // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds + // an open port. But we still need to tell our spark apps the right port to use. So + // only if the yarn config has the port set to 0, we prefer the value in the spark config conf.get("spark.shuffle.service.port").toInt } else { - t + tmpPort } } From 12b6a354eb0a60ee880b6546a7d7dc35538b1621 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Aug 2015 09:47:51 -0500 Subject: [PATCH 32/43] save registered executors when apps are removed; add tests --- .../shuffle/ExternalShuffleBlockResolver.java | 8 +++ .../yarn/YarnShuffleIntegrationSuite.scala | 13 +++-- .../yarn/YarnShuffleServiceSuite.scala | 50 +++++++++++++++++-- .../spark/network/yarn/YarnTestAccessor.scala | 4 +- 4 files changed, 63 insertions(+), 12 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index db396e0a70f6e..475069aa4fabf 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -182,6 +182,14 @@ public void run() { } } } + synchronized (executors) { + try { + saveRegisteredExecutors(); + } catch (Exception e) { + logger.error("Error saving registered executors", e); + } + } + } /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 5175048d1f01e..299164d616b36 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -45,14 +45,15 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { val shuffleServicePort = YarnTestAccessor.getShuffleServicePort val shuffleService = YarnTestAccessor.getShuffleServiceInstance - assert(!YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) + val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService) + assert(!registeredExecFile.exists()) logInfo("Shuffle service port = " + shuffleServicePort) val result = File.createTempFile("result", null, tempDir) runSpark( false, mainClassName(YarnExternalShuffleDriver.getClass), - appArgs = Seq(result.getAbsolutePath()), + appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath), extraConf = Map( "spark.shuffle.service.enabled" -> "true", "spark.shuffle.service.port" -> shuffleServicePort.toString @@ -60,7 +61,6 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { ) checkResult(result) assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) - assert(!YarnTestAccessor.loadSavedExecutors(shuffleService).isEmpty) } } @@ -69,13 +69,13 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 def main(args: Array[String]): Unit = { - if (args.length != 1) { + if (args.length != 2) { // scalastyle:off println System.err.println( s""" |Invalid command line: ${args.mkString(" ")} | - |Usage: ExternalShuffleDriver [result file] + |Usage: ExternalShuffleDriver [result file] [registed exec file] """.stripMargin) // scalastyle:on println System.exit(1) @@ -85,6 +85,8 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { .setAppName("External Shuffle Test")) val conf = sc.getConf val status = new File(args(0)) + val registeredExecFile = new File(args(1)) + logInfo("shuffle service executor file = " + registeredExecFile) var result = "failure" try { val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. @@ -92,6 +94,7 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet) result = "success" + assert(!YarnTestAccessor.loadSavedExecutors(registeredExecFile).isEmpty) } finally { sc.stop() Files.write(result, status, UTF_8) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index d03b287b853c1..5d1735748073a 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -23,7 +23,7 @@ import scala.annotation.tailrec import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext +import org.apache.hadoop.yarn.server.api.{ApplicationTerminationContext, ApplicationInitializationContext} import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.SparkFunSuite @@ -54,10 +54,17 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd var s3: YarnShuffleService = null override def afterEach(): Unit = { - Seq(s1, s2, s3).foreach { service => - if (service != null) { - service.stop() - } + if (s1 != null) { + s1.stop() + s1 = null + } + if (s2 != null) { + s2.stop() + s2 = null + } + if (s3 != null) { + s3.stop() + s3 = null } } @@ -148,6 +155,39 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd s3.stop() } + test("removed applications should not be in registered executor file") { + s1 = new YarnShuffleService + s1.init(yarnConfig) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + s1.initializeApplication(app1Data) + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + s1.initializeApplication(app2Data) + + val execStateFile = s1.registeredExecutorFile + execStateFile should not be (null) + execStateFile.exists() should be (false) + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") + + val blockHandler = s1.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + + YarnShuffleService.reloadRegisteredExecutors(execStateFile) should not be empty + + s1.stopApplication(new ApplicationTerminationContext(app1Id)) + s1.stopApplication(new ApplicationTerminationContext(app2Id)) + + YarnShuffleService.reloadRegisteredExecutors(execStateFile) shouldBe empty + } + test("shuffle service should be robust to corrupt registered executor file") { s1 = new YarnShuffleService diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala index ef29b03d35e1c..ce85d376c0215 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -41,9 +41,9 @@ object YarnTestAccessor { } def loadSavedExecutors( - service: YarnShuffleService + file: File ): JMap[String, JList[Entry[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo]]] = { - YarnShuffleService.reloadRegisteredExecutors(service.registeredExecutorFile) + YarnShuffleService.reloadRegisteredExecutors(file) } } From 79922b788e174edeaa030401e8e7c0d7ed034b12 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Aug 2015 15:16:11 -0500 Subject: [PATCH 33/43] rely on yarn to call stopApplication; assorted cleanup --- .../shuffle/ExternalShuffleBlockResolver.java | 49 +++++++++-- .../network/yarn/YarnShuffleService.java | 85 +------------------ .../yarn/YarnShuffleIntegrationSuite.scala | 9 +- .../network/shuffle/ShuffleTestAccessor.scala | 25 +++++- .../yarn/YarnShuffleServiceSuite.scala | 35 ++------ .../spark/network/yarn/YarnTestAccessor.scala | 6 -- 6 files changed, 87 insertions(+), 122 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 475069aa4fabf..2e73dad0235ab 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -18,8 +18,7 @@ package org.apache.spark.network.shuffle; import java.io.*; -import java.util.Iterator; -import java.util.Map; +import java.util.*; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.Executors; @@ -64,7 +63,8 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final File registeredExecutorFile; - private final DB db; + @VisibleForTesting + final DB db; public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( @@ -83,19 +83,24 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF if (registeredExecutorFile != null) { Options options = new Options(); options.createIfMissing(true); + options.logger(new LevelDBLogger()); JniDBFactory factory = new JniDBFactory(); DB tmpDb; + ConcurrentMap tmpExecutors; try { tmpDb = factory.open(registeredExecutorFile, options); - } catch (IOException e) { + tmpExecutors = reloadRegisteredExecutors(tmpDb); + } catch (Exception e) { logger.info("Error opening leveldb file {}", registeredExecutorFile, e); tmpDb = null; + tmpExecutors = Maps.newConcurrentMap(); } db = tmpDb; + executors = tmpExecutors; } else { db = null; + executors = Maps.newConcurrentMap(); } - this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; } @@ -316,4 +321,38 @@ private void saveRegisteredExecutors() throws IOException { db.put("registeredExecutors".getBytes(Charsets.UTF_8), bytes.toByteArray()); } } + + @VisibleForTesting + static ConcurrentMap reloadRegisteredExecutors(DB db) + throws IOException, ClassNotFoundException { + if (db != null) { + ObjectInputStream in = null; + byte[] bytes = db.get("registeredExecutors".getBytes(Charsets.UTF_8)); + if (bytes != null) { + try { + in = new ObjectInputStream(new ByteArrayInputStream(bytes)); + ConcurrentMap registeredExecutors = + (ConcurrentMap) in.readObject(); + in.close(); + return registeredExecutors; + } finally { + if (in != null) { + in.close(); + } + } + } + } + return Maps.newConcurrentMap(); + } + + private static class LevelDBLogger implements org.iq80.leveldb.Logger { + private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); + + @Override + public void log(String message) { + LOG.info(message); + } + } + + } diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 1634a89c9980a..6e499d2fa86b6 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -23,9 +23,7 @@ import java.util.Map.Entry; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.collect.Lists; -import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.AuxiliaryService; @@ -35,10 +33,6 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.fusesource.leveldbjni.JniDBFactory; -import org.fusesource.leveldbjni.internal.NativeDB.DBException; -import org.iq80.leveldb.DB; -import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,9 +98,6 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting static YarnShuffleService instance; - @VisibleForTesting - Map>> recoveredExecutorRegistrations = null; - public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); @@ -130,21 +121,11 @@ protected void serviceInit(Configuration conf) { // In case this NM was killed while there were running spark applications, we need to restore // lost state for the existing executors. We look for an existing file in the NM's local dirs. - // If we don't find one, then we choose a file to use to save the state next time. However, we - // do *not* immediately register all the executors in that file, just in case the application - // was terminated while the NM was restarting. We wait until yarn tells the service about the - // app again via #initializeApplication, so we know its still running. That is important - // for preventing a leak where the app data would stick around *forever*. This does leave - // a small race -- if the NM restarts *again*, after only some of the existing apps have been - // re-registered, the info of the remaining apps is lost. + // If we don't find one, then we choose a file to use to save the state next time. Even if + // an application was stopped while the NM was down, we expect yarn to call stopApplication() + // when it comes back registeredExecutorFile = findRegisteredExecutorFile(conf.getStrings("yarn.nodemanager.local-dirs")); - try { - recoveredExecutorRegistrations = reloadRegisteredExecutors(); - } catch (Exception e) { - recoveredExecutorRegistrations = new HashMap<>(); - logger.error("Failed to load previously registered executors", e); - } TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); // If authentication is enabled, set up the shuffle server to use a @@ -187,16 +168,6 @@ public void initializeApplication(ApplicationInitializationContext context) { } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } - // See if we already have data for this app from before the restart -- if so, re-register - // those executors - List> executorsForApp = - recoveredExecutorRegistrations.get(appId); - if (executorsForApp != null) { - for (Entry entry: executorsForApp) { - logger.info("re-registering {} with {}", entry.getKey(), entry.getValue()); - blockHandler.reregisterExecutor(entry.getKey(), entry.getValue()); - } - } } @Override @@ -257,54 +228,4 @@ protected void serviceStop() { public ByteBuffer getMetaData() { return ByteBuffer.allocate(0); } - - private Map>> reloadRegisteredExecutors() - throws IOException, ClassNotFoundException { - if (registeredExecutorFile != null && registeredExecutorFile.exists()) { - logger.info("Reloading executors from {}", registeredExecutorFile); - return reloadRegisteredExecutors(registeredExecutorFile); - } else { - logger.info("No executor info to reload"); - return new HashMap<>(); - } - } - - - @VisibleForTesting - static Map>> reloadRegisteredExecutors( - File file) throws IOException, ClassNotFoundException { - Map>> result = new HashMap<>(); - Options options = new Options(); - options.createIfMissing(true); - JniDBFactory factory = new JniDBFactory(); - DB db = null; - try { - db = factory.open(file, options); - ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream( - db.get("registeredExecutors".getBytes(Charsets.UTF_8)))); - Map registeredExecutors = - (Map) in.readObject(); - for (Entry e: registeredExecutors.entrySet()) { - List> executorsForApp = - result.get(e.getKey().appId); - if (executorsForApp == null) { - executorsForApp = new ArrayList<>(); - result.put(e.getKey().appId, executorsForApp); - } - executorsForApp.add(new AbstractMap.SimpleImmutableEntry<>(e.getKey(), e.getValue())); - } - in.close(); - } catch (DBException dbe) { - // blow the corrupt db away, so we can still write out OK data for future executors - FileUtils.deleteDirectory(file); - throw dbe; - } finally { - if (db != null) { - db.close(); - } - } - return result; - } - - } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 299164d616b36..5e8238822b90a 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -21,10 +21,12 @@ import java.io.File import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files +import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.apache.spark._ +import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} /** @@ -46,7 +48,6 @@ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { val shuffleService = YarnTestAccessor.getShuffleServiceInstance val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService) - assert(!registeredExecFile.exists()) logInfo("Shuffle service port = " + shuffleServicePort) val result = File.createTempFile("result", null, tempDir) @@ -88,15 +89,19 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { val registeredExecFile = new File(args(1)) logInfo("shuffle service executor file = " + registeredExecFile) var result = "failure" + val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup") try { val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. collect().toSet sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet) result = "success" - assert(!YarnTestAccessor.loadSavedExecutors(registeredExecFile).isEmpty) + // only one process can open a leveldb file at a time, so we copy the files + FileUtils.copyDirectory(registeredExecFile, execStateCopy) + assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty) } finally { sc.stop() + FileUtils.deleteDirectory(execStateCopy) Files.write(result, status, UTF_8) } } diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala index 97da440d636bc..aa46ec5100f0e 100644 --- a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala @@ -16,9 +16,13 @@ */ package org.apache.spark.network.shuffle -import java.io.File +import java.io.{IOException, File} +import java.util.concurrent.ConcurrentMap +import com.google.common.annotations.VisibleForTesting import org.apache.hadoop.yarn.api.records.ApplicationId +import org.fusesource.leveldbjni.JniDBFactory +import org.iq80.leveldb.{DB, Options} import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo @@ -45,4 +49,23 @@ object ShuffleTestAccessor { resolver.registeredExecutorFile } + def shuffleServiceLevelDB(resolver: ExternalShuffleBlockResolver): DB = { + resolver.db + } + + def reloadRegisteredExecutors( + file: File): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = { + val options: Options = new Options + options.createIfMissing(true) + val factory = new JniDBFactory + val db = factory.open(file, options) + val result = ExternalShuffleBlockResolver.reloadRegisteredExecutors(db) + db.close() + result + } + + def reloadRegisteredExecutors( + db: DB): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = { + ExternalShuffleBlockResolver.reloadRegisteredExecutors(db) + } } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 5d1735748073a..aef8c29a64ae9 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -116,16 +116,11 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd val handler2 = s2.blockHandler val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) - // until we initialize the application, don't know about any executors - // that is so that if the application gets removed while the NM was down, it still eventually - // gets purged from our list of apps. - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (None) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) - - // now we reinitialize only one of the apps (as if app2 was stopped during the NM restart) + // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped + // during the restart s2.initializeApplication(app1Data) + s2.stopApplication(new ApplicationTerminationContext(app2Id)) ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1)) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) // Act like the NM restarts one more time @@ -134,22 +129,11 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd s3.init(yarnConfig) s3.registeredExecutorFile should be (execStateFile) - // the second app won't even be in our file of saved executor info - // (this is mostly an implementation detail, by itself its not really an important check ...) - s3.recoveredExecutorRegistrations.get(app1Id.toString) should not be (null) - s3.recoveredExecutorRegistrations.get(app2Id.toString) should be (null) - val handler3 = s3.blockHandler val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (None) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) - - // now if we initialize both those apps, we'll have restored executor info for app 1, - // but for app 2, there won't anything to restore + // app1 is still running s3.initializeApplication(app1Data) - s3.initializeApplication(app2Data) - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) s3.stop() @@ -169,7 +153,6 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd val execStateFile = s1.registeredExecutorFile execStateFile should not be (null) - execStateFile.exists() should be (false) val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") @@ -180,16 +163,16 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) - YarnShuffleService.reloadRegisteredExecutors(execStateFile) should not be empty + val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver) + ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty s1.stopApplication(new ApplicationTerminationContext(app1Id)) + ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty s1.stopApplication(new ApplicationTerminationContext(app2Id)) - - YarnShuffleService.reloadRegisteredExecutors(execStateFile) shouldBe empty + ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty } - - test("shuffle service should be robust to corrupt registered executor file") { + ignore("shuffle service should be robust to corrupt registered executor file") { s1 = new YarnShuffleService s1.init(yarnConfig) val app1Id = ApplicationId.newInstance(0, 1) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala index ce85d376c0215..b80ae1e48d0f0 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -40,10 +40,4 @@ object YarnTestAccessor { service.registeredExecutorFile } - def loadSavedExecutors( - file: File - ): JMap[String, JList[Entry[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo]]] = { - YarnShuffleService.reloadRegisteredExecutors(file) - } - } From acedb620eed8d5bbb09b3fc3d998e574d7c6340f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Aug 2015 16:41:41 -0500 Subject: [PATCH 34/43] switch to writing out one record per executor --- .../shuffle/ExternalShuffleBlockResolver.java | 92 ++++++++++--------- .../network/yarn/YarnShuffleService.java | 13 +-- 2 files changed, 54 insertions(+), 51 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 2e73dad0235ab..44b7bf28843a2 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -28,7 +28,9 @@ import com.google.common.base.Objects; import com.google.common.collect.Maps; import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB; import org.iq80.leveldb.DB; +import org.iq80.leveldb.DBIterator; import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +68,7 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final DB db; - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) { + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) throws IOException { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); @@ -77,18 +79,31 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF ExternalShuffleBlockResolver( TransportConf conf, File registeredExecutorFile, - Executor directoryCleaner) { + Executor directoryCleaner) throws IOException { this.conf = conf; this.registeredExecutorFile = registeredExecutorFile; if (registeredExecutorFile != null) { Options options = new Options(); - options.createIfMissing(true); + options.createIfMissing(false); options.logger(new LevelDBLogger()); - JniDBFactory factory = new JniDBFactory(); DB tmpDb; ConcurrentMap tmpExecutors; try { - tmpDb = factory.open(registeredExecutorFile, options); + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException e) { + if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { + logger.info("Creating state database at " + registeredExecutorFile); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } else { + throw e; + } + } + try { tmpExecutors = reloadRegisteredExecutors(tmpDb); } catch (Exception e) { logger.info("Error opening leveldb file {}", registeredExecutorFile, e); @@ -114,7 +129,13 @@ public void registerExecutor( synchronized (executors) { executors.put(fullId, executorInfo); try { - saveRegisteredExecutors(); + ByteArrayOutputStream bytesOut = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bytesOut); + out.writeObject(executorInfo); + out.close(); + if (db != null) { + db.put(dbAppExecKey(new AppExecId(appId, execId)), bytesOut.toByteArray()); + } } catch (Exception e) { logger.error("Error saving registered executors", e); } @@ -173,6 +194,9 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { // Only touch executors associated with the appId that was removed. if (appId.equals(fullId.appId)) { it.remove(); + if (db != null) { + db.delete(dbAppExecKey(fullId)); + } if (cleanupLocalDirs) { logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length); @@ -187,14 +211,6 @@ public void run() { } } } - synchronized (executors) { - try { - saveRegisteredExecutors(); - } catch (Exception e) { - logger.error("Error saving registered executors", e); - } - } - } /** @@ -308,41 +324,35 @@ public String toString() { } } - /** - * write out the set of registered executors to a file so we can reload them on restart. - * You must have a lock on executors when calling this - */ - private void saveRegisteredExecutors() throws IOException { - if (db != null) { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - out.writeObject(executors); - out.close(); - db.put("registeredExecutors".getBytes(Charsets.UTF_8), bytes.toByteArray()); - } + private static byte[] dbAppExecKey(AppExecId appExecId) { + return (appExecId.appId + ";" + appExecId.execId).getBytes(Charsets.UTF_8); + } + + private static AppExecId parseDbAppExecKey(byte[] bytes) { + String s = new String(bytes, Charsets.UTF_8); + int p = s.indexOf(';'); + return new AppExecId(s.substring(0, p), s.substring(p + 1)); } @VisibleForTesting static ConcurrentMap reloadRegisteredExecutors(DB db) throws IOException, ClassNotFoundException { + ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); if (db != null) { - ObjectInputStream in = null; - byte[] bytes = db.get("registeredExecutors".getBytes(Charsets.UTF_8)); - if (bytes != null) { - try { - in = new ObjectInputStream(new ByteArrayInputStream(bytes)); - ConcurrentMap registeredExecutors = - (ConcurrentMap) in.readObject(); - in.close(); - return registeredExecutors; - } finally { - if (in != null) { - in.close(); - } - } + DBIterator itr = db.iterator(); + itr.seekToFirst(); + while (itr.hasNext()) { + Map.Entry e = itr.next(); + ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(e.getValue())); + AppExecId id = parseDbAppExecKey(e.getKey()); + registeredExecutors.put( + id, + (ExecutorShuffleInfo) in.readObject() + ); + in.close(); } } - return Maps.newConcurrentMap(); + return registeredExecutors; } private static class LevelDBLogger implements org.iq80.leveldb.Logger { diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 6e499d2fa86b6..a9615df08f5cd 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -17,22 +17,15 @@ package org.apache.spark.network.yarn; -import java.io.*; +import java.io.File; import java.nio.ByteBuffer; -import java.util.*; -import java.util.Map.Entry; +import java.util.List; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.server.api.AuxiliaryService; -import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; -import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; -import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; -import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; -import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.hadoop.yarn.server.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 9378ba39e3c8efbf2caf9fc1f71143fe0b2b01b3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Aug 2015 16:55:50 -0500 Subject: [PATCH 35/43] fail gracefully on corrupt leveldb files --- .../shuffle/ExternalShuffleBlockResolver.java | 14 +++++++++++++- .../spark/network/yarn/YarnShuffleService.java | 2 +- .../network/yarn/YarnShuffleServiceSuite.scala | 2 +- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 44b7bf28843a2..20531105780f2 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -100,7 +100,19 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF throw new IOException("Unable to create state store", dbExc); } } else { - throw e; + logger.error("error opening leveldb file {}. Creating new file, will not be able to " + + "recover state for existing applications", registeredExecutorFile, e); + for (File f: registeredExecutorFile.listFiles()) { + f.delete(); + } + registeredExecutorFile.delete(); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } } try { diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index a9615df08f5cd..11ea7f3fd3cfe 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -127,7 +127,7 @@ protected void serviceInit(Configuration conf) { try { blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); } catch (Exception e) { - logger.error("Failed to initial external shuffle service", e); + logger.error("Failed to initialize external shuffle service", e); } List bootstraps = Lists.newArrayList(); diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index aef8c29a64ae9..60d298165e139 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -172,7 +172,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty } - ignore("shuffle service should be robust to corrupt registered executor file") { + test("shuffle service should be robust to corrupt registered executor file") { s1 = new YarnShuffleService s1.init(yarnConfig) val app1Id = ApplicationId.newInstance(0, 1) From e9f99e82420f2bf435342d430b319d707a70e8c9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Aug 2015 17:05:59 -0500 Subject: [PATCH 36/43] cleanup the handling of bad dbs a little --- .../shuffle/ExternalShuffleBlockResolver.java | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 20531105780f2..54eccb654b305 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -87,7 +87,6 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF options.createIfMissing(false); options.logger(new LevelDBLogger()); DB tmpDb; - ConcurrentMap tmpExecutors; try { tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); } catch (NativeDB.DBException e) { @@ -100,6 +99,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF throw new IOException("Unable to create state store", dbExc); } } else { + // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new + // one, so we can keep processing new apps logger.error("error opening leveldb file {}. Creating new file, will not be able to " + "recover state for existing applications", registeredExecutorFile, e); for (File f: registeredExecutorFile.listFiles()) { @@ -115,15 +116,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF } } - try { - tmpExecutors = reloadRegisteredExecutors(tmpDb); - } catch (Exception e) { - logger.info("Error opening leveldb file {}", registeredExecutorFile, e); - tmpDb = null; - tmpExecutors = Maps.newConcurrentMap(); - } + executors = reloadRegisteredExecutors(tmpDb); db = tmpDb; - executors = tmpExecutors; } else { db = null; executors = Maps.newConcurrentMap(); @@ -348,7 +342,7 @@ private static AppExecId parseDbAppExecKey(byte[] bytes) { @VisibleForTesting static ConcurrentMap reloadRegisteredExecutors(DB db) - throws IOException, ClassNotFoundException { + throws IOException { ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); if (db != null) { DBIterator itr = db.iterator(); @@ -357,10 +351,14 @@ static ConcurrentMap reloadRegisteredExecutors(D Map.Entry e = itr.next(); ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(e.getValue())); AppExecId id = parseDbAppExecKey(e.getKey()); - registeredExecutors.put( - id, - (ExecutorShuffleInfo) in.readObject() - ); + try { + registeredExecutors.put( + id, + (ExecutorShuffleInfo) in.readObject() + ); + } catch (ClassNotFoundException e1) { + throw new IOException(e1); + } in.close(); } } From 8267d2a005e36c0af6d940594f76129756959bc4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Aug 2015 21:33:56 -0500 Subject: [PATCH 37/43] style --- .../spark/network/shuffle/ExternalShuffleBlockResolver.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 54eccb654b305..39a1f5f3881e0 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -68,7 +68,8 @@ public class ExternalShuffleBlockResolver { @VisibleForTesting final DB db; - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) throws IOException { + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) + throws IOException { this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); From 1a7980b672e249d5bef44954e6b4a683c9684850 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 7 Aug 2015 09:38:13 -0500 Subject: [PATCH 38/43] version --- .../shuffle/ExternalShuffleBlockResolver.java | 83 +++++++++++++++++-- 1 file changed, 77 insertions(+), 6 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 39a1f5f3881e0..9b904d580dd46 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -117,6 +117,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF } } + // if there is a version mismatch, we throw an exception, which means the service is unusable + checkVersion(tmpDb); executors = reloadRegisteredExecutors(tmpDb); db = tmpDb; } else { @@ -332,26 +334,31 @@ public String toString() { } private static byte[] dbAppExecKey(AppExecId appExecId) { - return (appExecId.appId + ";" + appExecId.execId).getBytes(Charsets.UTF_8); + return (APP_KEY_PREFIX + ";" + appExecId.appId + ";" + appExecId.execId).getBytes(Charsets.UTF_8); } - private static AppExecId parseDbAppExecKey(byte[] bytes) { - String s = new String(bytes, Charsets.UTF_8); + private static AppExecId parseDbAppExecKey(String s) { int p = s.indexOf(';'); - return new AppExecId(s.substring(0, p), s.substring(p + 1)); + int p2 = s.indexOf(';', p + 1); + return new AppExecId(s.substring(p + 1, p2), s.substring(p2 + 1)); } + private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; + @VisibleForTesting static ConcurrentMap reloadRegisteredExecutors(DB db) throws IOException { ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); if (db != null) { DBIterator itr = db.iterator(); - itr.seekToFirst(); + itr.seek(APP_KEY_PREFIX.getBytes(Charsets.UTF_8)); while (itr.hasNext()) { Map.Entry e = itr.next(); + String key = new String(e.getKey(), Charsets.UTF_8); + if (!key.startsWith(APP_KEY_PREFIX)) + break; + AppExecId id = parseDbAppExecKey(key); ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(e.getValue())); - AppExecId id = parseDbAppExecKey(e.getKey()); try { registeredExecutors.put( id, @@ -375,5 +382,69 @@ public void log(String message) { } } + private static final StoreVersion CURRENT_VERSION = new StoreVersion(1,0); + private static void checkVersion(DB db) throws IOException { + byte[] bytes = db.get(StoreVersion.KEY); + if (bytes == null) { + storeVersion(db); + } else if (bytes.length != 8) { + throw new IOException("unexpected version format"); + } else { + DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes)); + int major = in.readInt(); + int minor = in.readInt(); + StoreVersion version = new StoreVersion(major, minor); + if (version.major != CURRENT_VERSION.major) { + throw new IOException("cannot read state DB with version " + version + ", incompatible " + + "with current version " + CURRENT_VERSION); + } + storeVersion(db); + } + } + + private static void storeVersion(DB db) throws IOException { + ByteArrayOutputStream bytesOut = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(bytesOut); + out.writeInt(CURRENT_VERSION.major); + out.writeInt(CURRENT_VERSION.minor); + out.close(); + db.put(StoreVersion.KEY, bytesOut.toByteArray()); + } + + + private static class StoreVersion { + + final static byte[] KEY = "StoreVersion".getBytes(Charsets.UTF_8); + + final int major; + final int minor; + + StoreVersion(int major, int minor) { + this.major = major; + this.minor = minor; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StoreVersion that = (StoreVersion) o; + + if (major != that.major) return false; + if (minor != that.minor) return false; + + return true; + } + + @Override + public int hashCode() { + int result = major; + result = 31 * result + minor; + return result; + } + } + + } From 594d520d1e9faabf59da7df38a362f93c6a40d6e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 7 Aug 2015 13:55:22 -0500 Subject: [PATCH 39/43] use json to serialize application executor info --- network/shuffle/pom.xml | 5 ++ .../shuffle/ExternalShuffleBlockHandler.java | 2 +- .../shuffle/ExternalShuffleBlockResolver.java | 53 ++++++++++--------- .../shuffle/protocol/ExecutorShuffleInfo.java | 10 +++- .../ExternalShuffleBlockResolverSuite.java | 30 ++++++++++- .../shuffle/ExternalShuffleSecuritySuite.java | 2 +- 6 files changed, 73 insertions(+), 29 deletions(-) diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index 420d726e63b04..c53505a2435b1 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -49,6 +49,11 @@ 1.8 + + com.fasterxml.jackson.core + jackson-databind + + org.slf4j diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index e8b25b8f9af98..0df1dd621f6e9 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -51,7 +51,7 @@ public class ExternalShuffleBlockHandler extends RpcHandler { final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) { + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 9b904d580dd46..225bfc82a5c72 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -23,6 +23,9 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; import com.google.common.base.Objects; @@ -138,12 +141,10 @@ public void registerExecutor( synchronized (executors) { executors.put(fullId, executorInfo); try { - ByteArrayOutputStream bytesOut = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytesOut); - out.writeObject(executorInfo); - out.close(); if (db != null) { - db.put(dbAppExecKey(new AppExecId(appId, execId)), bytesOut.toByteArray()); + byte[] key = dbAppExecKey(new AppExecId(appId, execId)); + byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); + db.put(key, value); } } catch (Exception e) { logger.error("Error saving registered executors", e); @@ -204,7 +205,11 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { if (appId.equals(fullId.appId)) { it.remove(); if (db != null) { - db.delete(dbAppExecKey(fullId)); + try { + db.delete(dbAppExecKey(fullId)); + } catch (IOException e) { + logger.error("Error deleting {} from executor state db", appId, e); + } } if (cleanupLocalDirs) { @@ -301,11 +306,12 @@ void close() { } /** Simply encodes an executor's full ID, which is appId + execId. */ - public static class AppExecId implements Serializable { + public static class AppExecId { public final String appId; - final String execId; + public final String execId; - public AppExecId(String appId, String execId) { + @JsonCreator + public AppExecId(@JsonProperty("appId") String appId, @JsonProperty("execId") String execId) { this.appId = appId; this.execId = execId; } @@ -333,14 +339,20 @@ public String toString() { } } - private static byte[] dbAppExecKey(AppExecId appExecId) { - return (APP_KEY_PREFIX + ";" + appExecId.appId + ";" + appExecId.execId).getBytes(Charsets.UTF_8); + static ObjectMapper mapper = new ObjectMapper(); + + private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { + // we stick a common prefix on all the keys so we can find them in the DB + String appExecJson = mapper.writeValueAsString(appExecId); + String key = (APP_KEY_PREFIX + ";" + appExecJson); + return key.getBytes(Charsets.UTF_8); } - private static AppExecId parseDbAppExecKey(String s) { + private static AppExecId parseDbAppExecKey(String s) throws IOException { int p = s.indexOf(';'); - int p2 = s.indexOf(';', p + 1); - return new AppExecId(s.substring(p + 1, p2), s.substring(p2 + 1)); + String json = s.substring(p + 1); + AppExecId parsed = mapper.readValue(json, AppExecId.class); + return parsed; } private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; @@ -358,16 +370,9 @@ static ConcurrentMap reloadRegisteredExecutors(D if (!key.startsWith(APP_KEY_PREFIX)) break; AppExecId id = parseDbAppExecKey(key); - ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(e.getValue())); - try { - registeredExecutors.put( - id, - (ExecutorShuffleInfo) in.readObject() - ); - } catch (ClassNotFoundException e1) { - throw new IOException(e1); - } - in.close(); + ExecutorShuffleInfo shuffleInfo = + mapper.readValue(new String(e.getValue(), Charsets.UTF_8), ExecutorShuffleInfo.class); + registeredExecutors.put(id, shuffleInfo); } } return registeredExecutors; diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index 71b45dd5c80cf..73fa94acf2105 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.Arrays; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; @@ -27,7 +29,7 @@ import org.apache.spark.network.protocol.Encoders; /** Contains all configuration necessary for locating the shuffle files of an executor. */ -public class ExecutorShuffleInfo implements Encodable, Serializable { +public class ExecutorShuffleInfo implements Encodable { /** The base set of local directories that the executor stores its shuffle files in. */ public final String[] localDirs; /** Number of subdirectories created within each localDir. */ @@ -35,7 +37,11 @@ public class ExecutorShuffleInfo implements Encodable, Serializable { /** Shuffle manager (SortShuffleManager or HashShuffleManager) that the executor is using. */ public final String shuffleManager; - public ExecutorShuffleInfo(String[] localDirs, int subDirsPerLocalDir, String shuffleManager) { + @JsonCreator + public ExecutorShuffleInfo( + @JsonProperty("localDirs")String[] localDirs, + @JsonProperty("subDirsPerLocalDir") int subDirsPerLocalDir, + @JsonProperty("shuffleManager") String shuffleManager) { this.localDirs = localDirs; this.subDirsPerLocalDir = subDirsPerLocalDir; this.shuffleManager = shuffleManager; diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 808a554a774fd..9ec30854157f8 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -22,8 +22,10 @@ import java.io.InputStreamReader; import com.google.common.io.CharStreams; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -59,7 +61,7 @@ public static void afterAll() { } @Test - public void testBadRequests() { + public void testBadRequests() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { @@ -126,4 +128,30 @@ public void testHashShuffleBlocks() throws IOException { block1Stream.close(); assertEquals(hashBlock1, block1); } + + @Test + public void jsonSerializationOfExecutorRegistration() throws IOException { + AppExecId appId = new AppExecId("foo", "bar"); + String appIdJson = ExternalShuffleBlockResolver.mapper.writeValueAsString(appId); + AppExecId parsedAppId = + ExternalShuffleBlockResolver.mapper.readValue(appIdJson, AppExecId.class); + assertEquals(parsedAppId, appId); + + ExecutorShuffleInfo shuffleInfo = + new ExecutorShuffleInfo(new String[]{"/bippy", "/flippy"}, 7, "hash"); + String shuffleJson = ExternalShuffleBlockResolver.mapper.writeValueAsString(shuffleInfo); + ExecutorShuffleInfo parsedShuffleInfo = + ExternalShuffleBlockResolver.mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); + assertEquals(parsedShuffleInfo, shuffleInfo); + + // Intentionally keep these hard-coded strings in here, to check backwards-compatability. + // its not legacy yet, but keeping this here in case anybody changes it + String legacyAppIdJson = "{\"appId\":\"foo\", \"execId\":\"bar\"}"; + assertEquals(appId, + ExternalShuffleBlockResolver.mapper.readValue(legacyAppIdJson, AppExecId.class)); + String legacyShuffleJson = "{\"localDirs\": [\"/bippy\", \"/flippy\"], " + + "\"subDirsPerLocalDir\": 7, \"shuffleManager\": \"hash\"}"; + assertEquals(shuffleInfo, + ExternalShuffleBlockResolver.mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); + } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 5da708d325cf8..aa99efda94948 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -43,7 +43,7 @@ public class ExternalShuffleSecuritySuite { TransportServer server; @Before - public void beforeEach() { + public void beforeEach() throws IOException { TransportContext context = new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, From 795d28fd99690fb7ce0babddccec73be7c64b8b3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 12 Aug 2015 11:02:44 -0500 Subject: [PATCH 40/43] review feedback --- .../shuffle/ExternalShuffleBlockResolver.java | 49 +++++++------------ .../ExternalShuffleBlockResolverSuite.java | 17 +++---- .../yarn/YarnShuffleServiceSuite.scala | 7 ++- 3 files changed, 30 insertions(+), 43 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 225bfc82a5c72..d9c0719cbc469 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -57,6 +57,10 @@ public class ExternalShuffleBlockResolver { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); + private static final ObjectMapper mapper = new ObjectMapper(); + private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; + private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); + // Map containing all registered executors' metadata. @VisibleForTesting final ConcurrentMap executors; @@ -339,8 +343,6 @@ public String toString() { } } - static ObjectMapper mapper = new ObjectMapper(); - private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { // we stick a common prefix on all the keys so we can find them in the DB String appExecJson = mapper.writeValueAsString(appExecId); @@ -355,8 +357,6 @@ private static AppExecId parseDbAppExecKey(String s) throws IOException { return parsed; } - private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; - @VisibleForTesting static ConcurrentMap reloadRegisteredExecutors(DB db) throws IOException { @@ -367,11 +367,11 @@ static ConcurrentMap reloadRegisteredExecutors(D while (itr.hasNext()) { Map.Entry e = itr.next(); String key = new String(e.getKey(), Charsets.UTF_8); - if (!key.startsWith(APP_KEY_PREFIX)) + if (!key.startsWith(APP_KEY_PREFIX)) { break; + } AppExecId id = parseDbAppExecKey(key); - ExecutorShuffleInfo shuffleInfo = - mapper.readValue(new String(e.getValue(), Charsets.UTF_8), ExecutorShuffleInfo.class); + ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); registeredExecutors.put(id, shuffleInfo); } } @@ -387,18 +387,17 @@ public void log(String message) { } } - private static final StoreVersion CURRENT_VERSION = new StoreVersion(1,0); + /** + * Simple major.minor versioning scheme. Any incompatible changes should be across major + * versions. Minor version differences are allowed -- meaning we should be able to read + * dbs that are either earlier *or* later on the minor version. + */ private static void checkVersion(DB db) throws IOException { byte[] bytes = db.get(StoreVersion.KEY); if (bytes == null) { storeVersion(db); - } else if (bytes.length != 8) { - throw new IOException("unexpected version format"); } else { - DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes)); - int major = in.readInt(); - int minor = in.readInt(); - StoreVersion version = new StoreVersion(major, minor); + StoreVersion version = mapper.readValue(bytes, StoreVersion.class); if (version.major != CURRENT_VERSION.major) { throw new IOException("cannot read state DB with version " + version + ", incompatible " + "with current version " + CURRENT_VERSION); @@ -408,23 +407,18 @@ private static void checkVersion(DB db) throws IOException { } private static void storeVersion(DB db) throws IOException { - ByteArrayOutputStream bytesOut = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(bytesOut); - out.writeInt(CURRENT_VERSION.major); - out.writeInt(CURRENT_VERSION.minor); - out.close(); - db.put(StoreVersion.KEY, bytesOut.toByteArray()); + db.put(StoreVersion.KEY, mapper.writeValueAsBytes(CURRENT_VERSION)); } - private static class StoreVersion { + public static class StoreVersion { final static byte[] KEY = "StoreVersion".getBytes(Charsets.UTF_8); - final int major; - final int minor; + public final int major; + public final int minor; - StoreVersion(int major, int minor) { + @JsonCreator public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { this.major = major; this.minor = minor; } @@ -436,10 +430,7 @@ public boolean equals(Object o) { StoreVersion that = (StoreVersion) o; - if (major != that.major) return false; - if (minor != that.minor) return false; - - return true; + return major == that.major && minor == that.minor; } @Override @@ -450,6 +441,4 @@ public int hashCode() { } } - - } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 9ec30854157f8..3c6cb367dea46 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -21,6 +21,7 @@ import java.io.InputStream; import java.io.InputStreamReader; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CharStreams; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; @@ -131,27 +132,25 @@ public void testHashShuffleBlocks() throws IOException { @Test public void jsonSerializationOfExecutorRegistration() throws IOException { + ObjectMapper mapper = new ObjectMapper(); AppExecId appId = new AppExecId("foo", "bar"); - String appIdJson = ExternalShuffleBlockResolver.mapper.writeValueAsString(appId); - AppExecId parsedAppId = - ExternalShuffleBlockResolver.mapper.readValue(appIdJson, AppExecId.class); + String appIdJson = mapper.writeValueAsString(appId); + AppExecId parsedAppId = mapper.readValue(appIdJson, AppExecId.class); assertEquals(parsedAppId, appId); ExecutorShuffleInfo shuffleInfo = new ExecutorShuffleInfo(new String[]{"/bippy", "/flippy"}, 7, "hash"); - String shuffleJson = ExternalShuffleBlockResolver.mapper.writeValueAsString(shuffleInfo); + String shuffleJson = mapper.writeValueAsString(shuffleInfo); ExecutorShuffleInfo parsedShuffleInfo = - ExternalShuffleBlockResolver.mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); + mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); assertEquals(parsedShuffleInfo, shuffleInfo); // Intentionally keep these hard-coded strings in here, to check backwards-compatability. // its not legacy yet, but keeping this here in case anybody changes it String legacyAppIdJson = "{\"appId\":\"foo\", \"execId\":\"bar\"}"; - assertEquals(appId, - ExternalShuffleBlockResolver.mapper.readValue(legacyAppIdJson, AppExecId.class)); + assertEquals(appId, mapper.readValue(legacyAppIdJson, AppExecId.class)); String legacyShuffleJson = "{\"localDirs\": [\"/bippy\", \"/flippy\"], " + "\"subDirsPerLocalDir\": 7, \"shuffleManager\": \"hash\"}"; - assertEquals(shuffleInfo, - ExternalShuffleBlockResolver.mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); + assertEquals(shuffleInfo, mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); } } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 60d298165e139..2f22cbdbeac37 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -16,14 +16,14 @@ */ package org.apache.spark.network.yarn -import java.io.{DataOutputStream, FileOutputStream, PrintWriter, File} +import java.io.{DataOutputStream, File, FileOutputStream} import scala.annotation.tailrec import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.api.{ApplicationTerminationContext, ApplicationInitializationContext} +import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.SparkFunSuite @@ -33,11 +33,10 @@ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { private[yarn] var yarnConfig: YarnConfiguration = new YarnConfiguration - override def beforeEach(): Unit = { yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), - classOf[YarnShuffleService].getCanonicalName); + classOf[YarnShuffleService].getCanonicalName) yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => val d = new File(dir) From 2499c8c788201d541273bc70d858c079b0564662 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 12 Aug 2015 14:09:56 -0500 Subject: [PATCH 41/43] explicit dependency on jackson-annotations --- network/shuffle/pom.xml | 5 +++++ pom.xml | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index c53505a2435b1..3d2edf9d94515 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -54,6 +54,11 @@ jackson-databind + + com.fasterxml.jackson.core + jackson-annotations + + org.slf4j diff --git a/pom.xml b/pom.xml index cfd7d32563f2a..26f4ffdf5af6a 100644 --- a/pom.xml +++ b/pom.xml @@ -655,6 +655,11 @@ jackson-databind ${fasterxml.jackson.version} + + com.fasterxml.jackson.core + jackson-annotations + ${fasterxml.jackson.version} + From 5c71c8c9188cdf684cbd445c09eec54edae10e81 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 14 Aug 2015 11:46:40 -0500 Subject: [PATCH 42/43] save executor to db before registering; style --- .../shuffle/ExternalShuffleBlockResolver.java | 22 ++++++++++--------- .../shuffle/protocol/ExecutorShuffleInfo.java | 3 +-- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index d9c0719cbc469..994ca3acf217a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -58,6 +58,10 @@ public class ExternalShuffleBlockResolver { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); private static final ObjectMapper mapper = new ObjectMapper(); + /** + * This a common prefix to the key for each app registration we stick in leveldb, so they + * are easy to find, since leveldb lets you search based on prefix. + */ private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); @@ -142,18 +146,16 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); - synchronized (executors) { - executors.put(fullId, executorInfo); - try { - if (db != null) { - byte[] key = dbAppExecKey(new AppExecId(appId, execId)); - byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); - db.put(key, value); - } - } catch (Exception e) { - logger.error("Error saving registered executors", e); + try { + if (db != null) { + byte[] key = dbAppExecKey(fullId); + byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); + db.put(key, value); } + } catch (Exception e) { + logger.error("Error saving registered executors", e); } + executors.put(fullId, executorInfo); } /** diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index 73fa94acf2105..102d4efb8bf3b 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -17,7 +17,6 @@ package org.apache.spark.network.shuffle.protocol; -import java.io.Serializable; import java.util.Arrays; import com.fasterxml.jackson.annotation.JsonCreator; @@ -39,7 +38,7 @@ public class ExecutorShuffleInfo implements Encodable { @JsonCreator public ExecutorShuffleInfo( - @JsonProperty("localDirs")String[] localDirs, + @JsonProperty("localDirs") String[] localDirs, @JsonProperty("subDirsPerLocalDir") int subDirsPerLocalDir, @JsonProperty("shuffleManager") String shuffleManager) { this.localDirs = localDirs; From 0d285d3fac15afc77313255799a3392dcf74518f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 17 Aug 2015 16:11:39 -0500 Subject: [PATCH 43/43] review feedback --- .../shuffle/ExternalShuffleBlockResolver.java | 12 ++++++++---- .../apache/spark/network/yarn/YarnTestAccessor.scala | 6 ------ 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 994ca3acf217a..79beec4429a99 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -115,8 +115,10 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF // one, so we can keep processing new apps logger.error("error opening leveldb file {}. Creating new file, will not be able to " + "recover state for existing applications", registeredExecutorFile, e); - for (File f: registeredExecutorFile.listFiles()) { - f.delete(); + if (registeredExecutorFile.isDirectory()) { + for (File f : registeredExecutorFile.listFiles()) { + f.delete(); + } } registeredExecutorFile.delete(); options.createIfMissing(true); @@ -353,8 +355,10 @@ private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { } private static AppExecId parseDbAppExecKey(String s) throws IOException { - int p = s.indexOf(';'); - String json = s.substring(p + 1); + if (!s.startsWith(APP_KEY_PREFIX)) { + throw new IllegalArgumentException("expected a string starting with " + APP_KEY_PREFIX); + } + String json = s.substring(APP_KEY_PREFIX.length() + 1); AppExecId parsed = mapper.readValue(json, AppExecId.class); return parsed; } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala index b80ae1e48d0f0..db322cd18e150 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -17,12 +17,6 @@ package org.apache.spark.network.yarn import java.io.File -import java.util.{List => JList, Map => JMap} -import java.util.Map.Entry - - -import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo /** * just a cheat to get package-visible members in tests