From 791189b9f626ff4d0b59267521ac471f2c319ed1 Mon Sep 17 00:00:00 2001 From: RongGu Date: Mon, 17 Mar 2014 06:20:45 +0800 Subject: [PATCH 01/50] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark --- core/pom.xml | 21 +++ .../apache/spark/api/java/StorageLevels.java | 33 ++-- .../netty/TachyonFilePathResolver.java | 26 +++ .../scala/org/apache/spark/SparkContext.scala | 3 +- .../scala/org/apache/spark/SparkEnv.scala | 7 +- .../spark/deploy/worker/ExecutorRunner.scala | 1 + .../CoarseGrainedExecutorBackend.scala | 15 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../spark/executor/ExecutorExitCode.scala | 6 + .../cluster/SparkDeploySchedulerBackend.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 94 ++++++++-- .../spark/storage/BlockManagerMaster.scala | 5 +- .../storage/BlockManagerMasterActor.scala | 29 ++- .../spark/storage/BlockManagerMessages.scala | 16 +- .../apache/spark/storage/StorageLevel.scala | 54 +++--- .../apache/spark/storage/StorageUtils.scala | 21 ++- .../spark/storage/TachyonBlockManager.scala | 166 ++++++++++++++++++ .../spark/storage/TachyonFileSegment.scala | 31 ++++ .../apache/spark/storage/TachyonStore.scala | 144 +++++++++++++++ .../apache/spark/ui/storage/IndexPage.scala | 2 + .../scala/org/apache/spark/util/Utils.scala | 45 +++++ .../spark/storage/BlockManagerSuite.scala | 24 ++- .../spark/examples/SparkTachyonPi.scala | 58 ++++++ project/SparkBuild.scala | 6 + 24 files changed, 731 insertions(+), 84 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonStore.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala diff --git a/core/pom.xml b/core/pom.xml index a6f478b09bda0..2f16cd3c4bc38 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -205,6 +205,27 @@ commons-io test + + org.tachyonproject + tachyon + 0.4.0 + + + org.apache.hadoop + hadoop-client + + org.jboss.netty + netty + + org.apache.curator + curator-recipes + + + org.apache.curator + curator-test + + + org.scalatest scalatest_${scala.binary.version} diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 9f13b39909481..95be655b119eb 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -23,17 +23,19 @@ * Expose some commonly useful storage level constants. */ public class StorageLevels { - public static final StorageLevel NONE = create(false, false, false, 1); - public static final StorageLevel DISK_ONLY = create(true, false, false, 1); - public static final StorageLevel DISK_ONLY_2 = create(true, false, false, 2); - public static final StorageLevel MEMORY_ONLY = create(false, true, true, 1); - public static final StorageLevel MEMORY_ONLY_2 = create(false, true, true, 2); - public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, 1); - public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, 2); - public static final StorageLevel MEMORY_AND_DISK = create(true, true, true, 1); - public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, true, 2); - public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, 1); - public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, 2); + public static final StorageLevel NONE = new StorageLevel(false, false, false, false, 1); + public static final StorageLevel DISK_ONLY = new StorageLevel(true, false, false, false, 1); + public static final StorageLevel DISK_ONLY_2 = new StorageLevel(true, false, false, false, 2); + public static final StorageLevel MEMORY_ONLY = new StorageLevel(false, true, false, true, 1); + public static final StorageLevel MEMORY_ONLY_2 = new StorageLevel(false, true, false, true, 2); + public static final StorageLevel MEMORY_ONLY_SER = new StorageLevel(false, true, false, false, 1); + public static final StorageLevel MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, false, 2); + public static final StorageLevel MEMORY_AND_DISK = new StorageLevel(true, true, false, true, 1); + public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2); + public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false, 1); + public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2); + + public static final StorageLevel TACHYON = new StorageLevel(false, false, true, false, 1); /** * Create a new StorageLevel object. @@ -42,7 +44,12 @@ public class StorageLevels { * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ - public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { - return StorageLevel.apply(useDisk, useMemory, deserialized, replication); + public static StorageLevel create( + boolean useDisk, + boolean useMemory, + boolean useTachyon, + boolean deserialized, + int replication) { + return StorageLevel.apply(useDisk, useMemory, useTachyon, deserialized, replication); } } diff --git a/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java b/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java new file mode 100644 index 0000000000000..483de58c4976c --- /dev/null +++ b/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java @@ -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.netty; + +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.TachyonFileSegment; + +public interface TachyonFilePathResolver { + /** Get the file segment in which the given block resides. */ + TachyonFileSegment getBlockLocation(BlockId blockId); +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 852ed8fe1fb91..7f2756391fd97 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -139,7 +139,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, isDriver = true, - isLocal = isLocal) + isLocal = isLocal, + "" + appName) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5e43b5198422c..fd74081b0491e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -40,6 +40,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ class SparkEnv private[spark] ( val executorId: String, + val appId: String, val actorSystem: ActorSystem, val serializerManager: SerializerManager, val serializer: Serializer, @@ -121,7 +122,8 @@ object SparkEnv extends Logging { hostname: String, port: Int, isDriver: Boolean, - isLocal: Boolean): SparkEnv = { + isLocal: Boolean, + appId: String = null): SparkEnv = { val securityManager = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, @@ -169,7 +171,7 @@ object SparkEnv extends Logging { "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager) + serializer, conf, securityManager, appId) val connectionManager = blockManager.connectionManager @@ -219,6 +221,7 @@ object SparkEnv extends Logging { new SparkEnv( executorId, + appId, actorSystem, serializerManager, serializer, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2edd921066876..0e758a7e1f147 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -92,6 +92,7 @@ private[spark] class ExecutorRunner( def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl case "{{EXECUTOR_ID}}" => execId.toString + case "{{APP_ID}}" => appId.toString case "{{HOSTNAME}}" => host case "{{CORES}}" => cores.toString case other => other diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3486092a140fb..6d5963296d480 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,6 +31,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, executorId: String, + appId: String, hostPort: String, cores: Int) extends Actor @@ -53,7 +54,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + false, appId) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -92,7 +94,7 @@ private[spark] class CoarseGrainedExecutorBackend( } private[spark] object CoarseGrainedExecutorBackend { - def run(driverUrl: String, executorId: String, hostname: String, cores: Int, + def run(driverUrl: String, appId: String, executorId: String, hostname: String, cores: Int, workerUrl: Option[String]) { // Debug code Utils.checkHost(hostname) @@ -105,7 +107,8 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, appId, executorId, + sparkHostPort, cores), name = "Executor") workerUrl.foreach{ url => actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") @@ -118,13 +121,13 @@ private[spark] object CoarseGrainedExecutorBackend { case x if x < 4 => System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker - "Usage: CoarseGrainedExecutorBackend " + + "Usage: CoarseGrainedExecutorBackend " + " []") System.exit(1) case 4 => - run(args(0), args(1), args(2), args(3).toInt, None) + run(args(0), args(1), args(2), args(3), args(4).toInt, None) case x if x > 4 => - run(args(0), args(1), args(2), args(3).toInt, Some(args(4))) + run(args(0), args(1), args(2), args(3), args(4).toInt, Some(args(5))) } } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e69f6f72d3275..fae96ff245bcb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -38,7 +38,8 @@ private[spark] class Executor( executorId: String, slaveHostname: String, properties: Seq[(String, String)], - isLocal: Boolean = false) + isLocal: Boolean = false, + appId: String = null) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -103,7 +104,7 @@ private[spark] class Executor( private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, - isDriver = false, isLocal = false) + isDriver = false, isLocal = false, appId) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) _env diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 210f3dbeebaca..d9bad10507ffa 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -41,6 +41,12 @@ object ExecutorExitCode { /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 + /** TachyonStore failed to create a local temporary directory after many attempts. */ + val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 + + /** TachyonStore failed to create a local temporary directory after many attempts. */ + val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ee4b65e312abc..1152ff9e55acd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -45,7 +45,8 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{APP_ID}}", "{{HOSTNAME}}", + "{{CORES}}", "{{WORKER_URL}}") val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() 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 1bf3f4db32ea7..4e01b48aef6d5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -48,7 +48,8 @@ private[spark] class BlockManager( val defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager) + securityManager: SecurityManager, + appId: String = "test") extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) @@ -59,6 +60,9 @@ private[spark] class BlockManager( private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) + + var tachyonInitialized = false + private[storage] var innerTachyonStore: TachyonStore = null // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { @@ -97,6 +101,23 @@ private[spark] class BlockManager( var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object + private def tachyonStore : TachyonStore = synchronized { + if (!tachyonInitialized) { + initializeTachyonStore() + } + this.innerTachyonStore + } + + private def initializeTachyonStore() { + val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) + val tachyonStorePath = s"${storeDir}/${appId}/${this.executorId}" + val tachyonMaster = conf.get("spark.tachyonmaster.address", "localhost:19998") + val tachyonBlockManager = new TachyonBlockManager( + shuffleBlockManager, tachyonStorePath, tachyonMaster) + this.innerTachyonStore = new TachyonStore(this, tachyonBlockManager) + this.tachyonInitialized = true + } + private def heartBeat() { if (!master.sendHeartBeat(blockManagerId)) { reregister() @@ -123,9 +144,9 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf, securityManager: SecurityManager) = { + serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, appId: String) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, - securityManager) + securityManager, appId) } /** @@ -232,22 +253,26 @@ private[spark] class BlockManager( */ private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = { - val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { + val (curLevel, inMemSize, onDiskSize, inTachyonSize, tellMaster) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, false) + (StorageLevel.NONE, 0L, 0L, 0L, false) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) - val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) + val inTachyon = level.useTachyon && tachyonStore.contains(blockId) + val storageLevel = StorageLevel( + onDisk, inMem, inTachyon, level.deserialized, level.replication) val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize + val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else droppedMemorySize val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, info.tellMaster) + (storageLevel, memSize, diskSize, tachyonSize, info.tellMaster) } } if (tellMaster) { - master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) + master.updateBlockInfo( + blockManagerId, blockId, curLevel, inMemSize, onDiskSize, inTachyonSize) } else { true } @@ -330,6 +355,24 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } + + // Look for the block in Tachyon + if (level.useTachyon) { + logDebug("Getting block " + blockId + " from tachyon") + if (tachyonStore.contains(blockId)) { + tachyonStore.getBytes(blockId) match { + case Some(bytes) => { + if (!asValues) { + return Some(bytes) + } else { + return Some(dataDeserialize(blockId, bytes)) + } + } + case None => + logDebug("Block " + blockId + " not found in tachyon") + } + } + } // Look for block on disk, potentially storing it back into memory if required: if (level.useDisk) { @@ -575,7 +618,25 @@ private[spark] class BlockManager( case Right(newBytes) => bytesAfterPut = newBytes case Left(newIterator) => valuesAfterPut = newIterator } - } else { + } else if (level.useTachyon) { + // Save to Tachyon. + val askForBytes = level.replication > 1 + val res = data match { + case IteratorValues(iterator) => + tachyonStore.putValues(blockId, iterator, level, askForBytes) + case ArrayBufferValues(array) => + tachyonStore.putValues(blockId, array, level, askForBytes) + case ByteBufferValues(bytes) => { + bytes.rewind(); + tachyonStore.putBytes(blockId, bytes, level) + } + } + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case _ => + } + }else { // Save directly to disk. // Don't get back the bytes unless we replicate them. val askForBytes = level.replication > 1 @@ -658,7 +719,8 @@ private[spark] class BlockManager( */ var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { - val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) + val tLevel = StorageLevel( + level.useDisk, level.useMemory, level.useTachyon, level.deserialized, 1) if (cachedPeers == null) { cachedPeers = master.getPeers(blockManagerId, level.replication - 1) } @@ -760,9 +822,11 @@ private[spark] class BlockManager( // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) - if (!removedFromMemory && !removedFromDisk) { + val removedFromTachyon = + if (tachyonInitialized == true) tachyonStore.remove(blockId) else false + if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk or memory store") + "the disk, memory or tachyon store") } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { @@ -798,6 +862,9 @@ private[spark] class BlockManager( if (level.useDisk) { diskStore.remove(id) } + if (level.useTachyon) { + tachyonStore.remove(id) + } iterator.remove() logInfo("Dropped block " + id) } @@ -872,6 +939,9 @@ private[spark] class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() + if(tachyonInitialized == true) { + tachyonStore.clear() + } metadataCleaner.cancel() broadcastCleaner.cancel() logInfo("BlockManager stopped") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e531467cccb40..3e9b8fed01b7d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -65,9 +65,10 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): Boolean = { + diskSize: Long, + tachyonSize: Long): Boolean = { val res = askDriverWithReply[Boolean]( - UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) + UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) logInfo("Updated info of block " + blockId) res } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index a999d76a326a6..9685238088c6d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -72,10 +72,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act register(blockManagerId, maxMemSize, slaveActor) sender ! true - case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + case UpdateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => // TODO: Ideally we want to handle all the message replies in receive instead of in the // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) + updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -243,7 +244,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, + tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.executorId == "" && !isLocal) { @@ -262,7 +264,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act return } - blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) + blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize, tachyonSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { @@ -309,8 +311,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private[spark] object BlockManagerMasterActor { - - case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) + case class BlockStatus( + storageLevel: StorageLevel, memSize: Long, diskSize: Long, tachyonSize: Long) class BlockManagerInfo( val blockManagerId: BlockManagerId, @@ -333,7 +335,7 @@ object BlockManagerMasterActor { } def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, tachyonSize: Long) { updateLastSeenMs() @@ -353,17 +355,22 @@ object BlockManagerMasterActor { // They can be both larger than 0, when a block is dropped from memory to disk. // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0, 0)) _remainingMem -= memSize logInfo("Added %s in memory on %s (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), Utils.bytesToString(_remainingMem))) } if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize, 0)) logInfo("Added %s on disk on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } + if (storageLevel.useTachyon) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, tachyonSize)) + logInfo("Added %s on tachyon on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(tachyonSize))) + } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. val blockStatus: BlockStatus = _blocks.get(blockId) @@ -378,6 +385,10 @@ object BlockManagerMasterActor { logInfo("Removed %s on %s on disk (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } + if (blockStatus.storageLevel.useTachyon) { + logInfo("Removed %s on %s on tachyon (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize))) + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index bbb9529b5a0ca..436311828ae62 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -53,11 +53,12 @@ private[storage] object BlockManagerMessages { var blockId: BlockId, var storageLevel: StorageLevel, var memSize: Long, - var diskSize: Long) + var diskSize: Long, + var tachyonSize: Long) extends ToBlockManagerMaster with Externalizable { - def this() = this(null, null, null, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0, 0) // For deserialization only override def writeExternal(out: ObjectOutput) { blockManagerId.writeExternal(out) @@ -65,6 +66,7 @@ private[storage] object BlockManagerMessages { storageLevel.writeExternal(out) out.writeLong(memSize) out.writeLong(diskSize) + out.writeLong(tachyonSize) } override def readExternal(in: ObjectInput) { @@ -73,6 +75,7 @@ private[storage] object BlockManagerMessages { storageLevel = StorageLevel(in) memSize = in.readLong() diskSize = in.readLong() + tachyonSize = in.readLong() } } @@ -81,13 +84,14 @@ private[storage] object BlockManagerMessages { blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + diskSize: Long, + tachyonSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize) } // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1b7934d59fa1d..d43d602b1db90 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,8 +21,9 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, - * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory - * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. + * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to + * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on + * multiple nodes. * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). @@ -30,38 +31,41 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, + private var useTachyon_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. private def this(flags: Int, replication: Int) { - this((flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) + this((flags & 8) != 0, (flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) } - def this() = this(false, true, false) // For deserialization + def this() = this(false, true, false, false) // For deserialization def useDisk = useDisk_ def useMemory = useMemory_ + def useTachyon = useTachyon_ def deserialized = deserialized_ def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.deserialized, this.replication) + this.useDisk, this.useMemory, this.useTachyon, this.deserialized, this.replication) override def equals(other: Any): Boolean = other match { case s: StorageLevel => s.useDisk == useDisk && s.useMemory == useMemory && + s.useTachyon == useTachyon && s.deserialized == deserialized && s.replication == replication case _ => false } - def isValid = ((useMemory || useDisk) && (replication > 0)) + def isValid = ((useMemory || useDisk || useTachyon) && (replication > 0)) def toInt: Int = { var ret = 0 @@ -71,6 +75,9 @@ class StorageLevel private( if (useMemory_) { ret |= 2 } + if (useTachyon_) { + ret |= 2 + } if (deserialized_) { ret |= 1 } @@ -86,6 +93,7 @@ class StorageLevel private( val flags = in.readByte() useDisk_ = (flags & 4) != 0 useMemory_ = (flags & 2) != 0 + useTachyon_ = (flags & 2) != 0 deserialized_ = (flags & 1) != 0 replication_ = in.readByte() } @@ -93,14 +101,15 @@ class StorageLevel private( @throws(classOf[IOException]) private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) - override def toString: String = - "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) + override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( + useDisk, useMemory, useTachyon, deserialized, replication) override def hashCode(): Int = toInt * 41 + replication def description : String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") + result += (if (useTachyon) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") result += "%sx Replicated".format(replication) result @@ -113,21 +122,24 @@ class StorageLevel private( * new storage levels. */ object StorageLevel { - val NONE = new StorageLevel(false, false, false) - val DISK_ONLY = new StorageLevel(true, false, false) - val DISK_ONLY_2 = new StorageLevel(true, false, false, 2) - val MEMORY_ONLY = new StorageLevel(false, true, true) - val MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2) - val MEMORY_ONLY_SER = new StorageLevel(false, true, false) - val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2) - val MEMORY_AND_DISK = new StorageLevel(true, true, true) - val MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2) - val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) - val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) + val NONE = new StorageLevel(false, false, false, false) + val DISK_ONLY = new StorageLevel(true, false, false, false) + val DISK_ONLY_2 = new StorageLevel(true, false, false, false, 2) + val MEMORY_ONLY = new StorageLevel(false, true, false, true) + val MEMORY_ONLY_2 = new StorageLevel(false, true, false, true, 2) + val MEMORY_ONLY_SER = new StorageLevel(false, true, false, false) + val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, false, 2) + val MEMORY_AND_DISK = new StorageLevel(true, true, false, true) + val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) + val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) + val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) + + val TACHYON = new StorageLevel(false, false, true, false) /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = - getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) + def apply(useDisk: Boolean, useMemory: Boolean, useTachyon: Boolean, + deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int) = diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 2d88a40fbb3f2..baefed78fa773 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -44,13 +44,15 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) + numCachedPartitions: Int, numPartitions: Int, memSize: Long, tachyonSize: Long, diskSize: Long) extends Ordered[RDDInfo] { override def toString = { import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + - "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, bytesToString(memSize), bytesToString(diskSize)) + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + } override def compare(that: RDDInfo) = { @@ -87,14 +89,21 @@ object StorageUtils { val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) => // Add up memory and disk sizes val memSize = rddBlocks.map(_.memSize).reduce(_ + _) + val tachyonSize = rddBlocks.map(_.tachyonSize).reduce(_ + _) val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) // Get the friendly name and storage level for the RDD, if available sc.persistentRdds.get(rddId).map { r => val rddName = Option(r.name).getOrElse(rddId.toString) val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, - memSize, diskSize) + RDDInfo(rddId, + rddName, + rddStorageLevel, + rddBlocks.length, + r.partitions.size, + memSize, + tachyonSize, + diskSize) } }.flatten.toArray diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala new file mode 100644 index 0000000000000..2c0b309427f63 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -0,0 +1,166 @@ +/* + * 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.storage + +import java.io.File +import java.text.SimpleDateFormat +import java.util.{Date, Random} + +import tachyon.client.TachyonFS +import tachyon.client.TachyonFile + +import org.apache.spark.Logging +import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.network.netty.{TachyonFilePathResolver, ShuffleSender} +import org.apache.spark.util.Utils + + +/** + * Creates and maintains the logical mapping between logical blocks and tachyon fs + * locations. By default, one block is mapped to one file with a name given by its BlockId. + * However, it is also possible to have a block map to only a segment of a file, by calling + * mapBlockToFileSegment(). + * + * @param rootDirs The directories to use for storing block files. Data will be hashed among these. + */ +private[spark] class TachyonBlockManager( + shuffleManager: ShuffleBlockManager, + rootDirs: String, + val master: String) + extends TachyonFilePathResolver with Logging { + + val client = if (master != null && master != "") TachyonFS.get(master) else null + if (client == null) { + logError("Failed to connect to the Tachyon as the master address is not configured") + System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) + } + + private val MAX_DIR_CREATION_ATTEMPTS = 10 + private val subDirsPerTachyonDir = + shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt + + // Create one Tachyon directory for each path mentioned in spark.tachyon.dir; then, inside this + // directory, create multiple subdirectories that we will hash files into, in order to avoid + // having really large inodes at the top level in Tachyon. + private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() + private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) + + addShutdownHook() + + /** + * Returns the physical tachyon file segment in which the given BlockId is located. + * If the BlockId has been mapped to a specific FileSegment, that will be returned. + * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly. + */ + def getBlockLocation(blockId: BlockId): TachyonFileSegment = { + val file = getFile(blockId.name) + new TachyonFileSegment(file, 0, file.length()) + } + + def removeFile(file: TachyonFile): Boolean = { + client.delete(file.getPath(), false) + } + + def fileExists(file: TachyonFile): Boolean = { + client.exist(file.getPath()) + } + + def getFile(filename: String): TachyonFile = { + // Figure out which tachyon directory it hashes to, and which subdirectory in that + val hash = Utils.nonNegativeHash(filename) + val dirId = hash % tachyonDirs.length + val subDirId = (hash / tachyonDirs.length) % subDirsPerTachyonDir + + // Create the subdirectory if it doesn't already exist + var subDir = subDirs(dirId)(subDirId) + if (subDir == null) { + subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) + client.mkdir(path) + val newDir = client.getFile(path) + subDirs(dirId)(subDirId) = newDir + newDir + } + } + } + val filePath = subDir + "/" + filename + if(!client.exist(filePath)) { + client.createFile(filePath) + } + val file = client.getFile(filePath) + file + } + + def getFile(blockId: BlockId): TachyonFile = getFile(blockId.name) + + private def createTachyonDirs(): Array[TachyonFile] = { + logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") + val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + rootDirs.split(",").map { rootDir => + var foundLocalDir = false + var tachyonDir: TachyonFile = null + var tachyonDirId: String = null + var tries = 0 + val rand = new Random() + while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { + tries += 1 + try { + tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) + val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId + if (!client.exist(path)) { + foundLocalDir = client.mkdir(path) + tachyonDir = client.getFile(path) + } + } catch { + case e: Exception => + logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) + } + } + if (!foundLocalDir) { + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + + " attempts to create tachyon dir in " + rootDir) + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) + } + logInfo("Created tachyon directory at " + tachyonDir) + tachyonDir + } + } + + private def addShutdownHook() { + tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { + override def run() { + logDebug("Shutdown hook called") + tachyonDirs.foreach { tachyonDir => + try { + if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { + Utils.deleteRecursively(tachyonDir, client) + } + } catch { + case t: Throwable => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + } + } + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala new file mode 100644 index 0000000000000..bc336658c211b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala @@ -0,0 +1,31 @@ +/* + * 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.storage + +import java.io.File +import tachyon.client.TachyonFile + +/** + * References a particular segment of a file (potentially the entire file), + * based off an offset and a length. + */ + +private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) +{ + override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala new file mode 100644 index 0000000000000..f36f74d45de25 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -0,0 +1,144 @@ +/* + * 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.storage + +import java.io.RandomAccessFile +import java.nio.ByteBuffer +import java.util.LinkedHashMap +import java.io.FileOutputStream +import java.nio.channels.FileChannel.MapMode + +import scala.collection.mutable.ArrayBuffer + +import tachyon.client.OutStream +import tachyon.client.WriteType +import tachyon.client.ReadType +import tachyon.client.InStream + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.serializer.Serializer + + +private class Entry(val size: Long) +/** + * Stores BlockManager blocks on Tachyon. + */ +private class TachyonStore( + blockManager: BlockManager, + tachyonManager: TachyonBlockManager) + extends BlockStore(blockManager: BlockManager) with Logging { + + logInfo("TachyonStore started") + + override def getSize(blockId: BlockId): Long = { + tachyonManager.getBlockLocation(blockId).length + } + + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { + putToTachyonStore(blockId, _bytes, true) + } + + override def putValues( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + return putValues(blockId, values.toIterator, level, returnValues) + } + + override def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + logDebug("Attempting to write values for block " + blockId) + val _bytes = blockManager.dataSerialize(blockId, values) + putToTachyonStore(blockId, _bytes, returnValues) + } + + private def putToTachyonStore( + blockId: BlockId, + _bytes: ByteBuffer, + returnValues: Boolean): PutResult = { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val bytes = _bytes.duplicate() + bytes.rewind() + logDebug("Attempting to put block " + blockId + " into Tachyon") + val startTime = System.currentTimeMillis + val file = tachyonManager.getFile(blockId) + val os = file.getOutStream(WriteType.MUST_CACHE) + os.write(bytes.array()) + os.close() + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file in Tachyon in %d ms".format( + blockId, Utils.bytesToString(bytes.limit), (finishTime - startTime))) + + if (returnValues) { + PutResult(_bytes.limit(), Right(_bytes.duplicate())) + } else { + PutResult(_bytes.limit(), null) + } + } + + override def remove(blockId: BlockId): Boolean = { + val fileSegment = tachyonManager.getBlockLocation(blockId) + val file = fileSegment.file + if (tachyonManager.fileExists(file) && file.length() == fileSegment.length) { + tachyonManager.removeFile(file) + } else { + if (fileSegment.length < file.length()) { + logWarning("Could not delete block associated with only a part of a file: " + blockId) + } + false + } + } + + override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) + } + + /** + * A version of getValues that allows a custom serializer. This is used as part of the + * shuffle short-circuit code. + */ + def getValues(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { + getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer)) + } + + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + val segment = tachyonManager.getBlockLocation(blockId) + val file = tachyonManager.getFile(blockId) + val is = file.getInStream(ReadType.CACHE) + var buffer : ByteBuffer = null + if (is != null){ + val size = segment.length - segment.offset + val bs = new Array[Byte](size.asInstanceOf[Int]) + is.read(bs, segment.offset.asInstanceOf[Int] , size.asInstanceOf[Int]) + buffer = ByteBuffer.wrap(bs) + } + Some(buffer) + } + + override def contains(blockId: BlockId): Boolean = { + val fileSegment = tachyonManager.getBlockLocation(blockId) + val file = fileSegment.file + tachyonManager.fileExists(file) + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 6a3c41fb1155d..e5144aaf3a6b1 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -40,6 +40,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { "Cached Partitions", "Fraction Cached", "Size in Memory", + "Size in Tachyon", "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) val content = listingTable(rddHeaders, rddRow, rdds) @@ -59,6 +60,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} + {Utils.bytesToString(rdd.tachyonSize)} {Utils.bytesToString(rdd.diskSize)} } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 38a275d438959..2e1cf82feb1f7 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -37,6 +37,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil +import tachyon.client.TachyonFile +import tachyon.client.TachyonFS + /** * Various utility methods used by Spark. @@ -150,6 +153,7 @@ private[spark] object Utils extends Logging { } private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() + private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { @@ -158,6 +162,14 @@ private[spark] object Utils extends Logging { shutdownDeletePaths += absolutePath } } + + // Register the tachyon path to be deleted via shutdown hook + def registerShutdownDeleteDir(tachyonfile: TachyonFile) { + val absolutePath = tachyonfile.getPath() + shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths += absolutePath + } + } // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteDir(file: File): Boolean = { @@ -166,6 +178,14 @@ private[spark] object Utils extends Logging { shutdownDeletePaths.contains(absolutePath) } } + + // Is the path already registered to be deleted via a shutdown hook ? + def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths.contains(absolutePath) + } + } // Note: if file is child of some registered path, while not equal to it, then return true; // else false. This is to ensure that two shutdown hooks do not try to delete each others @@ -182,6 +202,22 @@ private[spark] object Utils extends Logging { } retval } + + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in Exception and incomplete cleanup. + def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + val retval = shutdownDeletePaths.synchronized { + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") + } + retval + } /** Create a temporary directory inside the given parent directory */ def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { @@ -533,6 +569,15 @@ private[spark] object Utils extends Logging { throw new IOException("Failed to delete: " + file) } } + + /** + * Delete a file or directory and its contents recursively. + */ + def deleteRecursively(dir: TachyonFile, client: TachyonFS) { + if (!client.delete(dir.getPath(), true)) { + throw new IOException("Failed to delete the tachyon dir: " + dir) + } + } /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1036b9f34e9dd..b66f3b894fff1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -94,9 +94,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("StorageLevel object caching") { - val level1 = StorageLevel(false, false, false, 3) - val level2 = StorageLevel(false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, 2) // this should return a different object + val level1 = StorageLevel(false, false, false, false, 3) + val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 + val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -407,6 +407,24 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store") assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store") } + + test("tachyon storage") { + val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) + if (tachyonUnitTestEnabled) { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("a1", a1, StorageLevel.TACHYON) + store.putSingle("a2", a2, StorageLevel.TACHYON) + store.putSingle("a3", a3, StorageLevel.TACHYON) + assert(store.getSingle("a2").isDefined, "a2 was in store") + assert(store.getSingle("a3").isDefined, "a3 was in store") + assert(store.getSingle("a1").isDefined, "a1 was in store") + } else { + info("tachyon storage test disabled.") + } + } test("on-disk storage") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala new file mode 100644 index 0000000000000..931da67e590b1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -0,0 +1,58 @@ +/* + * 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.examples + +import scala.math.random +import org.apache.spark._ +import SparkContext._ +import org.apache.spark.storage.StorageLevel + +/** Computes an approximation to pi */ +object SparkTachyonPi { + def main(args: Array[String]) { + if (args.length == 0) { + System.err.println("Usage: SparkTachyonPi []") + System.exit(1) + } + val spark = new SparkContext(args(0), "SparkTachyonPi", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + + val slices = if (args.length > 1) args(1).toInt else 2 + val n = 100000 * slices + + val rdd = spark.parallelize(1 to n, slices) + rdd.persist(StorageLevel.TACHYON) + val count = rdd.map { i => + val x = random * 2 - 1 + val y = random * 2 - 1 + if (x * x + y * y < 1) 1 else 0 + }.reduce(_ + _) + println("1- Pi is roughly " + 4.0 * count / n) + + val rdd2 = spark.parallelize(1 to n, slices) + rdd2.persist(StorageLevel.TACHYON) + val count2 = rdd2.map { i => + val x = random * 2 - 1 + val y = random * 2 - 1 + if (x * x + y * y < 1) 1 else 0 + }.reduce(_ + _) + println("2- Pi is roughly " + 4.0 * count2 / n) + + spark.stop() + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0c3bf29dfd4f..61bd164385723 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -269,6 +269,11 @@ object SparkBuild extends Build { val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") + val excludeKyro = ExclusionRule(organization = "de.javakaffee") + val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") + val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") + val excludeCurator = ExclusionRule(organization = "org.apache.curator") + def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -306,6 +311,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), + "org.tachyonproject" % "tachyon" % "0.4.0" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 556978b9393d751b61359c445d2caf13cf29aa66 Mon Sep 17 00:00:00 2001 From: RongGu Date: Tue, 18 Mar 2014 00:48:02 +0800 Subject: [PATCH 02/50] fix the scalastyle errors --- .../src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 3 ++- .../scala/org/apache/spark/storage/BlockManagerMessages.scala | 3 ++- 3 files changed, 5 insertions(+), 3 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 4e01b48aef6d5..78b14ce4d7f8f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -144,7 +144,7 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, appId: String) = { + serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, appId: String) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, securityManager, appId) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 9685238088c6d..6c4c70420fd71 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -264,7 +264,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act return } - blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize, tachyonSize) + blockManagerInfo(blockManagerId).updateBlockInfo( + blockId, storageLevel, memSize, diskSize, tachyonSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 436311828ae62..8a36b5cc42dfd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -90,7 +90,8 @@ private[storage] object BlockManagerMessages { } // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { + def unapply(h: UpdateBlockInfo) + : Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) } } From 70ca182aefa02b5a21fd02e7884f95377b4054be Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 21 Mar 2014 20:19:50 +0800 Subject: [PATCH 03/50] a bit change in comment --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 78b14ce4d7f8f..98eca4f04770a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -658,8 +658,8 @@ private[spark] class BlockManager( } } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. + // Now that the block is in either the memory, tachyon or disk store, + // let other threads read it, and tell the master about it. marked = true myInfo.markReady(size) if (tellMaster) { From 8011a96fb30f6689256904d072d41e17833bf038 Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 21 Mar 2014 23:34:39 +0800 Subject: [PATCH 04/50] fix a brought-in mistake in StorageLevel --- .../scala/org/apache/spark/storage/StorageLevel.scala | 8 ++++---- .../org/apache/spark/storage/BlockManagerSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index d43d602b1db90..6c4c1d2e40669 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -70,10 +70,10 @@ class StorageLevel private( def toInt: Int = { var ret = 0 if (useDisk_) { - ret |= 4 + ret |= 8 } if (useMemory_) { - ret |= 2 + ret |= 4 } if (useTachyon_) { ret |= 2 @@ -91,8 +91,8 @@ class StorageLevel private( override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk_ = (flags & 4) != 0 - useMemory_ = (flags & 2) != 0 + useDisk_ = (flags & 8) != 0 + useMemory_ = (flags & 4) != 0 useTachyon_ = (flags & 2) != 0 deserialized_ = (flags & 1) != 0 replication_ = in.readByte() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index b66f3b894fff1..8a962164f7645 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -156,12 +156,12 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(master.getLocations("a1").size === 0, "master did not remove a1") assert(master.getLocations("a2").size === 0, "master did not remove a2") } - + test("master + 2 managers interaction") { store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, securityMgr) - + val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") From e01a271afb91001b7950eabf737bb76dd768593e Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 22:25:40 -0700 Subject: [PATCH 05/50] update tachyon 0.4.1 --- core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 2f16cd3c4bc38..c8fd5966b0531 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -208,7 +208,7 @@ org.tachyonproject tachyon - 0.4.0 + 0.4.1 org.apache.hadoop From dc8ef24cb681a5e9fe6b13e469419e52ae6230ba Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 22:39:30 -0700 Subject: [PATCH 06/50] add old storelevel constructor --- .../main/java/org/apache/spark/api/java/StorageLevels.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 95be655b119eb..404dc4281d876 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -34,9 +34,13 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2); - + public static final StorageLevel TACHYON = new StorageLevel(false, false, true, false, 1); + public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { + return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); + } + /** * Create a new StorageLevel object. * @param useDisk saved to disk, if true From 47304b395f4e4ef4b276697489046f891c6cb58f Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 23:18:19 -0700 Subject: [PATCH 07/50] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. --- .../java/org/apache/spark/api/java/StorageLevels.java | 9 +++++++++ .../scala/org/apache/spark/storage/BlockManager.scala | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 404dc4281d876..db1b3005688b6 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -37,6 +37,14 @@ public class StorageLevels { public static final StorageLevel TACHYON = new StorageLevel(false, false, true, false, 1); + /** + * Create a new StorageLevel object. + * @param useDisk saved to disk, if true + * @param useMemory saved to memory, if true + * @param deserialized saved as deserialized objects, if true + * @param replication replication factor + */ + @Deprecated public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); } @@ -45,6 +53,7 @@ public static StorageLevel create(boolean useDisk, boolean useMemory, boolean de * Create a new StorageLevel object. * @param useDisk saved to disk, if true * @param useMemory saved to memory, if true + * @param useTachyon saved to Tachyon, if true * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ 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 98eca4f04770a..518396d98d7b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -101,7 +101,7 @@ private[spark] class BlockManager( var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object - private def tachyonStore : TachyonStore = synchronized { + private[storage] lazy val tachyonStore : TachyonStore = { if (!tachyonInitialized) { initializeTachyonStore() } From e554b1eec7db659bcc1138c320138b8ec3e2b3b3 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 23:27:42 -0700 Subject: [PATCH 08/50] add python code --- .../apache/spark/api/java/StorageLevels.java | 1 - python/pyspark/storagelevel.py | 24 ++++++++++--------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index db1b3005688b6..c19ee3360928f 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -34,7 +34,6 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2); - public static final StorageLevel TACHYON = new StorageLevel(false, false, true, false, 1); /** diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index c3e3a44e8e7ab..78af4d01a98b1 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -25,9 +25,10 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useTachyon, deserialized, replication = 1): self.useDisk = useDisk self.useMemory = useMemory + self.useTachyon = useTachyon self.deserialized = deserialized self.replication = replication @@ -35,13 +36,14 @@ def __repr__(self): return "StorageLevel(%s, %s, %s, %s)" % ( self.useDisk, self.useMemory, self.deserialized, self.replication) -StorageLevel.DISK_ONLY = StorageLevel(True, False, False) -StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2) -StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True) -StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, True, 2) -StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False) -StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, 2) -StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, True) -StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, True, 2) -StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False) -StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, 2) +StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False) +StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2) +StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, True) +StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, True, 2) +StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False, False) +StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, False, 2) +StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, True) +StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) +StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) +StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) +StorageLevel.TACHYON = StorageLevel(False, False, True, False, 1) \ No newline at end of file From fcaeab2b608b5b42828d239ad041a45c78c57bfb Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 23:29:53 -0700 Subject: [PATCH 09/50] address Aaron's comment --- .../apache/spark/api/java/StorageLevels.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index c19ee3360928f..9922bc1dad38e 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -23,18 +23,18 @@ * Expose some commonly useful storage level constants. */ public class StorageLevels { - public static final StorageLevel NONE = new StorageLevel(false, false, false, false, 1); - public static final StorageLevel DISK_ONLY = new StorageLevel(true, false, false, false, 1); - public static final StorageLevel DISK_ONLY_2 = new StorageLevel(true, false, false, false, 2); - public static final StorageLevel MEMORY_ONLY = new StorageLevel(false, true, false, true, 1); - public static final StorageLevel MEMORY_ONLY_2 = new StorageLevel(false, true, false, true, 2); - public static final StorageLevel MEMORY_ONLY_SER = new StorageLevel(false, true, false, false, 1); - public static final StorageLevel MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, false, 2); - public static final StorageLevel MEMORY_AND_DISK = new StorageLevel(true, true, false, true, 1); - public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2); - public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false, 1); - public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2); - public static final StorageLevel TACHYON = new StorageLevel(false, false, true, false, 1); + public static final StorageLevel NONE = create(false, false, false, false, 1); + public static final StorageLevel DISK_ONLY = create(true, false, false, false, 1); + public static final StorageLevel DISK_ONLY_2 = create(true, false, false, false, 2); + public static final StorageLevel MEMORY_ONLY = create(false, true, false, true, 1); + public static final StorageLevel MEMORY_ONLY_2 = create(false, true, false, true, 2); + public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, false, 1); + public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, false, 2); + public static final StorageLevel MEMORY_AND_DISK = create(true, true, false, true, 1); + public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); + public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); + public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); + public static final StorageLevel TACHYON = create(false, false, true, false, 1); /** * Create a new StorageLevel object. From e3ddbba931cf53c7d8024f668ded93be80402066 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 21 Mar 2014 23:52:30 -0700 Subject: [PATCH 10/50] add doc to use Tachyon cache mode. --- .../apache/spark/storage/StorageLevel.scala | 1 - docs/scala-programming-guide.md | 22 ++++++++++--------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 6c4c1d2e40669..a5b617ba23ace 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -133,7 +133,6 @@ object StorageLevel { val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) - val TACHYON = new StorageLevel(false, false, true, false) /** Create a new StorageLevel object */ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 99412733d4268..4fdbf8e8f4de4 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -23,7 +23,7 @@ To write a Spark application, you need to add a dependency on Spark. If you use groupId = org.apache.spark artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} + version = {{site.SPARK_VERSION}} In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: @@ -73,14 +73,14 @@ The master URL passed to Spark can be in one of the following formats: - - -
Master URLMeaning
local Run Spark locally with one worker thread (i.e. no parallelism at all).
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
spark://HOST:PORT Connect to the given Spark standalone - cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. +
spark://HOST:PORT Connect to the given Spark standalone + cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
mesos://HOST:PORT Connect to the given Mesos cluster. - The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, - which is 5050 by default. +
mesos://HOST:PORT Connect to the given Mesos cluster. + The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, + which is 5050 by default.
@@ -313,14 +313,16 @@ We recommend going through the following process to select one: * If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. * If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects - much more space-efficient, but still reasonably fast to access. + much more space-efficient, but still reasonably fast to access. You can also use `Tachyon` mode + to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will significantly + reduce JVM GC overhead. * Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk. * Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones let you continue running tasks on the RDD without waiting to recompute a lost partition. - -If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. + +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. # Shared Variables From 88593713f05f7b3b8f48de0fb29ba6a273d35d7b Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 00:18:44 -0700 Subject: [PATCH 11/50] various minor fixes and clean up --- .../apache/spark/storage/BlockManager.scala | 39 +++++++------------ .../apache/spark/storage/StorageUtils.scala | 19 +++------ .../spark/storage/TachyonBlockManager.scala | 7 ++-- .../spark/storage/TachyonFileSegment.scala | 9 ++--- .../apache/spark/storage/TachyonStore.scala | 8 +--- .../spark/storage/BlockManagerSuite.scala | 6 +-- .../org/apache/spark/examples/SparkPi.scala | 2 +- .../spark/examples/SparkTachyonPi.scala | 2 +- project/SparkBuild.scala | 2 +- 9 files changed, 34 insertions(+), 60 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 518396d98d7b4..d72f81720dc37 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -60,9 +60,16 @@ private[spark] class BlockManager( private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) - var tachyonInitialized = false - private[storage] var innerTachyonStore: TachyonStore = null + private[storage] lazy val tachyonStore : TachyonStore = { + val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) + val tachyonStorePath = s"${storeDir}/${appId}/${this.executorId}" + val tachyonMaster = conf.get("spark.tachyonmaster.address", "localhost:19998") + val tachyonBlockManager = new TachyonBlockManager( + shuffleBlockManager, tachyonStorePath, tachyonMaster) + tachyonInitialized = true + new TachyonStore(this, tachyonBlockManager) + } // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { @@ -101,23 +108,6 @@ private[spark] class BlockManager( var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object - private[storage] lazy val tachyonStore : TachyonStore = { - if (!tachyonInitialized) { - initializeTachyonStore() - } - this.innerTachyonStore - } - - private def initializeTachyonStore() { - val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) - val tachyonStorePath = s"${storeDir}/${appId}/${this.executorId}" - val tachyonMaster = conf.get("spark.tachyonmaster.address", "localhost:19998") - val tachyonBlockManager = new TachyonBlockManager( - shuffleBlockManager, tachyonStorePath, tachyonMaster) - this.innerTachyonStore = new TachyonStore(this, tachyonBlockManager) - this.tachyonInitialized = true - } - private def heartBeat() { if (!master.sendHeartBeat(blockManagerId)) { reregister() @@ -636,7 +626,7 @@ private[spark] class BlockManager( case Right(newBytes) => bytesAfterPut = newBytes case _ => } - }else { + } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. val askForBytes = level.replication > 1 @@ -658,7 +648,7 @@ private[spark] class BlockManager( } } - // Now that the block is in either the memory, tachyon or disk store, + // Now that the block is in either the memory, tachyon, or disk store, // let other threads read it, and tell the master about it. marked = true myInfo.markReady(size) @@ -822,11 +812,10 @@ private[spark] class BlockManager( // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) - val removedFromTachyon = - if (tachyonInitialized == true) tachyonStore.remove(blockId) else false + val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk, memory or tachyon store") + "the disk, memory, or tachyon store") } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { @@ -939,7 +928,7 @@ private[spark] class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() - if(tachyonInitialized == true) { + if(tachyonInitialized) { tachyonStore.clear() } metadataCleaner.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index baefed78fa773..e5a6d9b4404b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -48,11 +48,10 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, extends Ordered[RDDInfo] { override def toString = { import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + - "TachyonSize: %s; DiskSize: %s").format( - name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) - + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -96,14 +95,8 @@ object StorageUtils { sc.persistentRdds.get(rddId).map { r => val rddName = Option(r.name).getOrElse(rddId.toString) val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, - rddName, - rddStorageLevel, - rddBlocks.length, - r.partitions.size, - memSize, - tachyonSize, - diskSize) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, + tachyonSize, diskSize) } }.flatten.toArray diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 2c0b309427f63..e42e4981b7a43 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -17,7 +17,6 @@ package org.apache.spark.storage -import java.io.File import java.text.SimpleDateFormat import java.util.{Date, Random} @@ -47,7 +46,7 @@ private[spark] class TachyonBlockManager( val client = if (master != null && master != "") TachyonFS.get(master) else null if (client == null) { logError("Failed to connect to the Tachyon as the master address is not configured") - System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) + System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } private val MAX_DIR_CREATION_ATTEMPTS = 10 @@ -136,8 +135,8 @@ private[spark] class TachyonBlockManager( } } if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + - " attempts to create tachyon dir in " + rootDir) + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + + rootDir) System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) } logInfo("Created tachyon directory at " + tachyonDir) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala index bc336658c211b..b86abbda1d3e7 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala @@ -17,15 +17,12 @@ package org.apache.spark.storage -import java.io.File import tachyon.client.TachyonFile /** - * References a particular segment of a file (potentially the entire file), - * based off an offset and a length. + * References a particular segment of a file (potentially the entire file), based off an offset and + * a length. */ - -private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) -{ +private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index f36f74d45de25..193618bbec5b9 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -17,18 +17,12 @@ package org.apache.spark.storage -import java.io.RandomAccessFile import java.nio.ByteBuffer -import java.util.LinkedHashMap -import java.io.FileOutputStream -import java.nio.channels.FileChannel.MapMode import scala.collection.mutable.ArrayBuffer -import tachyon.client.OutStream import tachyon.client.WriteType import tachyon.client.ReadType -import tachyon.client.InStream import org.apache.spark.Logging import org.apache.spark.util.Utils @@ -36,6 +30,8 @@ import org.apache.spark.serializer.Serializer private class Entry(val size: Long) + + /** * Stores BlockManager blocks on Tachyon. */ diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 8a962164f7645..a150f3eddfdbb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -156,12 +156,12 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(master.getLocations("a1").size === 0, "master did not remove a1") assert(master.getLocations("a2").size === 0, "master did not remove a2") } - + test("master + 2 managers interaction") { store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, securityMgr) - + val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") @@ -407,7 +407,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store") assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store") } - + test("tachyon storage") { val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index e5a09ecec006f..d3babc3ed12c8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -18,8 +18,8 @@ package org.apache.spark.examples import scala.math.random + import org.apache.spark._ -import SparkContext._ /** Computes an approximation to pi */ object SparkPi { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 931da67e590b1..d5e43ab772ce8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -18,8 +18,8 @@ package org.apache.spark.examples import scala.math.random + import org.apache.spark._ -import SparkContext._ import org.apache.spark.storage.StorageLevel /** Computes an approximation to pi */ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 61bd164385723..6c4c59ba6e472 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -311,7 +311,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.0" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm), + "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 776a56c82eb0c8ae3dae688dc55818b28325e584 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 00:44:03 -0700 Subject: [PATCH 12/50] address patrick's and ali's comments from the previous PR --- .../spark/storage/TachyonBlockManager.scala | 11 ++- .../apache/spark/storage/TachyonStore.scala | 2 +- docs/scala-programming-guide.md | 75 +++++++++++++++---- 3 files changed, 65 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index e42e4981b7a43..026de9f028a78 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -30,10 +30,8 @@ import org.apache.spark.util.Utils /** - * Creates and maintains the logical mapping between logical blocks and tachyon fs - * locations. By default, one block is mapped to one file with a name given by its BlockId. - * However, it is also possible to have a block map to only a segment of a file, by calling - * mapBlockToFileSegment(). + * Creates and maintains the logical mapping between logical blocks and tachyon fs locations. By + * default, one block is mapped to one file with a name given by its BlockId. * * @param rootDirs The directories to use for storing block files. Data will be hashed among these. */ @@ -44,11 +42,12 @@ private[spark] class TachyonBlockManager( extends TachyonFilePathResolver with Logging { val client = if (master != null && master != "") TachyonFS.get(master) else null + if (client == null) { logError("Failed to connect to the Tachyon as the master address is not configured") - System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_INITIALIZE) } - + private val MAX_DIR_CREATION_ATTEMPTS = 10 private val subDirsPerTachyonDir = shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 193618bbec5b9..7248a4231bc7b 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -122,7 +122,7 @@ private class TachyonStore( val segment = tachyonManager.getBlockLocation(blockId) val file = tachyonManager.getFile(blockId) val is = file.getInStream(ReadType.CACHE) - var buffer : ByteBuffer = null + var buffer: ByteBuffer = null if (is != null){ val size = segment.length - segment.offset val bs = new Array[Byte](size.asInstanceOf[Int]) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 4fdbf8e8f4de4..62d6ffc7df6c9 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -265,11 +265,24 @@ A complete list of actions is available in the [RDD API doc](api/core/index.html ## RDD Persistence -One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory across operations. When you persist an RDD, each node stores any slices of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for building iterative algorithms with Spark and for interactive use from the interpreter. - -You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. - -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: +One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory +across operations. When you persist an RDD, each node stores any slices of it that it computes in +memory and reuses them in other actions on that dataset (or datasets derived from it). This allows +future actions to be much faster (often by more than 10x). Caching is a key tool for building +iterative algorithms with Spark and for interactive use from the interpreter. + +You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time +it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- +if any partition of an RDD is lost, it will automatically be recomputed using the transformations +that originally created it. + +In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to +persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), +or even replicate it across nodes. These levels are chosen by passing a +[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +object to `persist()`. The `cache()` method is a shorthand for using the default storage level, +which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of +available storage levels is: @@ -292,8 +305,8 @@ In addition, each RDD can be stored using a different *storage level*, allowing - + @@ -307,8 +320,9 @@ In addition, each RDD can be stored using a different *storage level*, allowing ### Which Storage Level to Choose? -Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency. -We recommend going through the following process to select one: +Spark's storage levels are meant to provide different trade-offs between memory usage and CPU +efficiency. It allows uses to choose memory, disk, or Tachyon for storing data. We recommend going +through the following process to select one: * If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. @@ -322,17 +336,38 @@ We recommend going through the following process to select one: application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones let you continue running tasks on the RDD without waiting to recompute a lost partition. -If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then +use the function factor method `apply()` of the +[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. + +Spark has a block manager inside the Executors that let you chose memory, disk, or Tachyon. The +latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system +[Tachyon](http://tachyon-project.org/). This mode has the following advantages: + +* Executor crash won't lose the data cached. +* Executors can have smaller memory footprint, allowing you to run more executors on the same +machine as the bulk of the memory will be inside Tachyon. +* There won't be GC overheads with data stored in Tachyon. # Shared Variables -Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a remote cluster node, it works on separate copies of all the variables used in the function. These variables are copied to each machine, and no updates to the variables on the remote machine are propagated back to the driver program. Supporting general, read-write shared variables across tasks would be inefficient. However, Spark does provide two limited types of *shared variables* for two common usage patterns: broadcast variables and accumulators. +Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a +remote cluster node, it works on separate copies of all the variables used in the function. These +variables are copied to each machine, and no updates to the variables on the remote machine are +propagated back to the driver program. Supporting general, read-write shared variables across tasks +would be inefficient. However, Spark does provide two limited types of *shared variables* for two +common usage patterns: broadcast variables and accumulators. ## Broadcast Variables -Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost. +Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather +than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a +large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables +using efficient broadcast algorithms to reduce communication cost. -Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The interpreter session below shows this: +Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The +broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` +method. The interpreter session below shows this: {% highlight scala %} scala> val broadcastVar = sc.broadcast(Array(1, 2, 3)) @@ -342,13 +377,21 @@ scala> broadcastVar.value res0: Array[Int] = Array(1, 2, 3) {% endhighlight %} -After the broadcast variable is created, it should be used instead of the value `v` in any functions run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later). +After the broadcast variable is created, it should be used instead of the value `v` in any functions +run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object +`v` should not be modified after it is broadcast in order to ensure that all nodes get the same +value of the broadcast variable (e.g. if the variable is shipped to a new node later). ## Accumulators -Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable collections, and programmers can add support for new types. +Accumulators are variables that are only "added" to through an associative operation and can +therefore be efficiently supported in parallel. They can be used to implement counters (as in +MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable +collections, and programmers can add support for new types. -An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method. +An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks +running on the cluster can then add to it using the `+=` operator. However, they cannot read its +value. Only the driver program can read the accumulator's value, using its `value` method. The interpreter session below shows an accumulator being used to add up the elements of an array: From e82909cab9454e12ed45f01e3f95dfcaa5d96655 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 01:10:12 -0700 Subject: [PATCH 13/50] minor cleanup --- .../src/main/java/org/apache/spark/api/java/StorageLevels.java | 3 ++- .../src/main/scala/org/apache/spark/storage/StorageLevel.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 9922bc1dad38e..0944f1089f5a3 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -44,7 +44,8 @@ public class StorageLevels { * @param replication replication factor */ @Deprecated - public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { + public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, + int replication) { return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index a5b617ba23ace..ddf9e1a08e3e4 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -138,7 +138,7 @@ object StorageLevel { /** Create a new StorageLevel object */ def apply(useDisk: Boolean, useMemory: Boolean, useTachyon: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( - new StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication)) + new StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int) = From bf278fa3c7ca2b53d16f51c89fe79c60be89599d Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 03:30:04 -0700 Subject: [PATCH 14/50] fix python tests --- python/pyspark/context.py | 2 +- python/pyspark/rdd.py | 3 ++- python/pyspark/storagelevel.py | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index bf2454fd7e38e..efd942e4a23bf 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -383,7 +383,7 @@ def _getJavaStorageLevel(self, storageLevel): raise Exception("storageLevel must be of type pyspark.StorageLevel") newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel - return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, + return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.userTachyon, storageLevel.deserialized, storageLevel.replication) def setJobGroup(self, groupId, description): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6d549b40e5698..e8aae4ecb665e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1178,11 +1178,12 @@ def getStorageLevel(self): Get the RDD's current storage level. >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.getStorageLevel() - StorageLevel(False, False, False, 1) + StorageLevel(False, False, False, False, 1) """ java_storage_level = self._jrdd.getStorageLevel() storage_level = StorageLevel(java_storage_level.useDisk(), java_storage_level.useMemory(), + java_storage_level.useTachyon(), java_storage_level.deserialized(), java_storage_level.replication()) return storage_level diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 78af4d01a98b1..1f5e877f9e589 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -33,8 +33,8 @@ def __init__(self, useDisk, useMemory, useTachyon, deserialized, replication = 1 self.replication = replication def __repr__(self): - return "StorageLevel(%s, %s, %s, %s)" % ( - self.useDisk, self.useMemory, self.deserialized, self.replication) + return "StorageLevel(%s, %s, %s, %s, %s)" % ( + self.useDisk, self.useMemory, self.useTachyon, self.deserialized, self.replication) StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False) StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2) From 1dcadf909bb3e0fe7f57258fb9b4f33923e0bd03 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 03:31:13 -0700 Subject: [PATCH 15/50] typo --- python/pyspark/context.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index efd942e4a23bf..5c98db235f3f8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -383,7 +383,7 @@ def _getJavaStorageLevel(self, storageLevel): raise Exception("storageLevel must be of type pyspark.StorageLevel") newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel - return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.userTachyon, + return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.useTachyon, storageLevel.deserialized, storageLevel.replication) def setJobGroup(self, groupId, description): From 77be7e81d8129827ee1b8ce968a25f51fd43f651 Mon Sep 17 00:00:00 2001 From: RongGu Date: Sun, 23 Mar 2014 10:38:44 +0800 Subject: [PATCH 16/50] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. --- .../main/scala/org/apache/spark/SparkContext.scala | 12 +++++++----- core/src/main/scala/org/apache/spark/SparkEnv.scala | 7 ++----- .../apache/spark/deploy/worker/ExecutorRunner.scala | 1 - .../executor/CoarseGrainedExecutorBackend.scala | 13 ++++++------- .../scala/org/apache/spark/executor/Executor.scala | 5 ++--- .../cluster/SparkDeploySchedulerBackend.scala | 3 +-- .../org/apache/spark/storage/BlockManager.scala | 12 ++++++------ 7 files changed, 24 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7f2756391fd97..4361ccc391bd0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -21,12 +21,10 @@ import java.io._ import java.net.URI import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger - import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} @@ -34,7 +32,6 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary - import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -45,6 +42,7 @@ import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import java.util.Random /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -127,6 +125,11 @@ class SparkContext( val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + + // Generate the random name for a temp folder in Tachyon + // Add a timestamp as the suffix here to make it more safe + val tachyonFolderName = new Random().nextInt() + "_" + System.currentTimeMillis() + conf.set("spark.tachyonstore.foldername", tachyonFolderName) val isLocal = (master == "local" || master.startsWith("local[")) @@ -139,8 +142,7 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, isDriver = true, - isLocal = isLocal, - "" + appName) + isLocal = isLocal) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index fd74081b0491e..5e43b5198422c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -40,7 +40,6 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ class SparkEnv private[spark] ( val executorId: String, - val appId: String, val actorSystem: ActorSystem, val serializerManager: SerializerManager, val serializer: Serializer, @@ -122,8 +121,7 @@ object SparkEnv extends Logging { hostname: String, port: Int, isDriver: Boolean, - isLocal: Boolean, - appId: String = null): SparkEnv = { + isLocal: Boolean): SparkEnv = { val securityManager = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, @@ -171,7 +169,7 @@ object SparkEnv extends Logging { "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, appId) + serializer, conf, securityManager) val connectionManager = blockManager.connectionManager @@ -221,7 +219,6 @@ object SparkEnv extends Logging { new SparkEnv( executorId, - appId, actorSystem, serializerManager, serializer, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 0e758a7e1f147..2edd921066876 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -92,7 +92,6 @@ private[spark] class ExecutorRunner( def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl case "{{EXECUTOR_ID}}" => execId.toString - case "{{APP_ID}}" => appId.toString case "{{HOSTNAME}}" => host case "{{CORES}}" => cores.toString case other => other diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 6d5963296d480..16887d8892b31 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,7 +31,6 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, executorId: String, - appId: String, hostPort: String, cores: Int) extends Actor @@ -55,7 +54,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Successfully registered with driver") // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, - false, appId) + false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -94,7 +93,7 @@ private[spark] class CoarseGrainedExecutorBackend( } private[spark] object CoarseGrainedExecutorBackend { - def run(driverUrl: String, appId: String, executorId: String, hostname: String, cores: Int, + def run(driverUrl: String, executorId: String, hostname: String, cores: Int, workerUrl: Option[String]) { // Debug code Utils.checkHost(hostname) @@ -107,7 +106,7 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, appId, executorId, + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") workerUrl.foreach{ url => @@ -121,13 +120,13 @@ private[spark] object CoarseGrainedExecutorBackend { case x if x < 4 => System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker - "Usage: CoarseGrainedExecutorBackend " + + "Usage: CoarseGrainedExecutorBackend " + " []") System.exit(1) case 4 => - run(args(0), args(1), args(2), args(3), args(4).toInt, None) + run(args(0), args(1), args(2), args(3).toInt, None) case x if x > 4 => - run(args(0), args(1), args(2), args(3), args(4).toInt, Some(args(5))) + run(args(0), args(1), args(2), args(3).toInt, Some(args(4))) } } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index fae96ff245bcb..e69f6f72d3275 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -38,8 +38,7 @@ private[spark] class Executor( executorId: String, slaveHostname: String, properties: Seq[(String, String)], - isLocal: Boolean = false, - appId: String = null) + isLocal: Boolean = false) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -104,7 +103,7 @@ private[spark] class Executor( private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, - isDriver = false, isLocal = false, appId) + isDriver = false, isLocal = false) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) _env diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 1152ff9e55acd..ee4b65e312abc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -45,8 +45,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{APP_ID}}", "{{HOSTNAME}}", - "{{CORES}}", "{{WORKER_URL}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() 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 d72f81720dc37..55be03a22c027 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -48,8 +48,7 @@ private[spark] class BlockManager( val defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager, - appId: String = "test") + securityManager: SecurityManager) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) @@ -63,8 +62,9 @@ private[spark] class BlockManager( var tachyonInitialized = false private[storage] lazy val tachyonStore : TachyonStore = { val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) - val tachyonStorePath = s"${storeDir}/${appId}/${this.executorId}" - val tachyonMaster = conf.get("spark.tachyonmaster.address", "localhost:19998") + val appFolderName = conf.get("spark.tachyonstore.foldername") + val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" + val tachyonMaster = conf.get("spark.tachyonmaster.address", "tachyon://localhost:19998") val tachyonBlockManager = new TachyonBlockManager( shuffleBlockManager, tachyonStorePath, tachyonMaster) tachyonInitialized = true @@ -134,9 +134,9 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, appId: String) = { + serializer: Serializer, conf: SparkConf, securityManager: SecurityManager) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, - securityManager, appId) + securityManager) } /** From 8968b6763f3c47a1c8f50caaa19c6611a5548cf2 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sat, 22 Mar 2014 23:25:23 -0700 Subject: [PATCH 17/50] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. --- core/pom.xml | 30 ++++++++++++++++++++++++++++-- project/SparkBuild.scala | 4 +++- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index c8fd5966b0531..f8224506a2c62 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -214,12 +214,38 @@ org.apache.hadoop hadoop-client - org.jboss.netty - netty org.apache.curator curator-recipes + + org.eclipse.jetty + jetty-jsp + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + junit + junit + + + org.powermock + powermock-module-junit4 + + + org.powermock + powermock-api-mockito + org.apache.curator curator-test diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6c4c59ba6e472..041cbcd6deb6e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -264,6 +264,7 @@ object SparkBuild extends Build { val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") @@ -273,6 +274,7 @@ object SparkBuild extends Build { val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") val excludeCurator = ExclusionRule(organization = "org.apache.curator") + val excludePowermock = ExclusionRule(organization = "org.powermock") def coreSettings = sharedSettings ++ Seq( @@ -311,7 +313,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeKyro, excludeHadoop, excludeCurator, excludeJackson, excludeNetty, excludeAsm), + "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 6a22c1ae9bb2cc52d03f69ab512193080b946ce1 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Sun, 23 Mar 2014 14:17:04 -0700 Subject: [PATCH 18/50] fix scalastyle --- .../storage/BlockManagerMasterActor.scala | 20 +++++++++---------- .../apache/spark/storage/StorageUtils.scala | 10 +++++----- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 8ad55791a2d6f..4f2e313827835 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -313,8 +313,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private[spark] case class BlockStatus( - storageLevel: StorageLevel, - memSize: Long, + storageLevel: StorageLevel, + memSize: Long, diskSize: Long, tachyonSize: Long) @@ -339,14 +339,14 @@ private[spark] class BlockManagerInfo( } def updateBlockInfo( - blockId: BlockId, - storageLevel: StorageLevel, + blockId: BlockId, + storageLevel: StorageLevel, memSize: Long, - diskSize: Long, + diskSize: Long, tachyonSize: Long) { - + updateLastSeenMs() - + if (_blocks.containsKey(blockId)) { // The block exists on the slave already. val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel @@ -355,7 +355,7 @@ private[spark] class BlockManagerInfo( _remainingMem += memSize } } - + if (storageLevel.isValid) { /* isValid means it is either stored in-memory or on-disk. * But the memSize here indicates the data size in or dropped from memory, @@ -391,8 +391,8 @@ private[spark] class BlockManagerInfo( Utils.bytesToString(_remainingMem))) } if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) + logInfo("Removed %s on %s on disk (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } if (blockStatus.storageLevel.useTachyon) { logInfo("Removed %s on %s on tachyon (size: %s)".format( diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 0332a1e022d04..ff6e84cf9819a 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -49,10 +49,10 @@ class StorageStatus( private[spark] class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L @@ -112,7 +112,7 @@ object StorageUtils { val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => // Add up memory, disk and Tachyon sizes - val persistedBlocks = + val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) From ca1446916fe23d898caec5bd1b82f5de1c09d154 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Mon, 24 Mar 2014 01:33:15 -0700 Subject: [PATCH 19/50] bump tachyon version to 0.4.1-thrift --- core/pom.xml | 2 +- project/SparkBuild.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index f8224506a2c62..746626378a8da 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -208,7 +208,7 @@ org.tachyonproject tachyon - 0.4.1 + 0.4.1-thrift org.apache.hadoop diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c502f73957412..ced7825a3b8a3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -188,8 +188,8 @@ object SparkBuild extends Build { "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", // For Sonatype publishing - //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", // also check the local Maven repository ~/.m2 Resolver.mavenLocal ), @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 716e93b81bdc2ad33d934ca76e591d8ac695a14d Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Mon, 24 Mar 2014 01:53:18 -0700 Subject: [PATCH 20/50] revert the version --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ced7825a3b8a3..774b90ae4d836 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From d827250b972d7f6d540dc5357d35ea269528ac2e Mon Sep 17 00:00:00 2001 From: RongGu Date: Tue, 25 Mar 2014 00:20:44 +0800 Subject: [PATCH 21/50] fix JsonProtocolSuie test failure --- .../org/apache/spark/util/JsonProtocol.scala | 2 ++ .../apache/spark/util/JsonProtocolSuite.scala | 18 +++++++++--------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 07a4ce773218f..77b4041195d43 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -599,11 +599,13 @@ private[spark] object JsonProtocol { val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] + val tachyonSize = (json \ "Tachyon Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 861224e144557..4f8ae3177aa64 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -471,19 +471,19 @@ class JsonProtocolSuite extends FunSuite { """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, - "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", - "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":200,"Number of Cached Partitions":300, + "Memory Size":400,"Disk Size":500,"Tachyon Size":0},"Emitted Task Size Warning":false}, + "Properties":{"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} """ private val stageCompletedJsonString = """ {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, - "Disk Size":501},"Emitted Task Size Warning":false}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, + "Memory Size":401,"Disk Size":501,"Tachyon Size":0},"Emitted Task Size Warning":false}} """ private val taskStartJsonString = @@ -516,8 +516,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = From bbeb4dec8f825b8d9c4dd4883e8e8ecb7c9097ee Mon Sep 17 00:00:00 2001 From: RongGu Date: Tue, 25 Mar 2014 08:48:28 +0800 Subject: [PATCH 22/50] fix the JsonProtocolSuite test failure problem --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 3 +++ project/SparkBuild.scala | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 77b4041195d43..99cbed29bb18a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -274,12 +274,14 @@ private[spark] object JsonProtocol { ("Number of Partitions" -> rddInfo.numPartitions) ~ ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ + ("Tachyon Size" -> rddInfo.tachyonSize) ~ ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ + ("Use Tachyon" -> storageLevel.useTachyon) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -317,6 +319,7 @@ private[spark] object JsonProtocol { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ ("Memory Size" -> blockStatus.memSize) ~ + ("Tachyon Size" -> blockStatus.tachyonSize) ~ ("Disk Size" -> blockStatus.diskSize) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ced7825a3b8a3..774b90ae4d836 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 16c579849b7a65e0bde97b9626d1e86571887f78 Mon Sep 17 00:00:00 2001 From: RongGu Date: Tue, 25 Mar 2014 09:28:18 +0800 Subject: [PATCH 23/50] make the dependency on tachyon as tachyon-0.4.1-thrift --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 774b90ae4d836..ced7825a3b8a3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 86a2eab3faf77239216ed901c6f9c12dfa674416 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Mon, 24 Mar 2014 11:15:59 -0700 Subject: [PATCH 24/50] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 --- core/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 746626378a8da..f8224506a2c62 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -208,7 +208,7 @@ org.tachyonproject tachyon - 0.4.1-thrift + 0.4.1 org.apache.hadoop diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ced7825a3b8a3..774b90ae4d836 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From 939e467e2cc4a05c27bfb265bb9248a6fc61c17f Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Mon, 24 Mar 2014 17:27:27 -0700 Subject: [PATCH 25/50] 0.4.1-thrift from maven central --- core/pom.xml | 2 +- project/SparkBuild.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index f8224506a2c62..746626378a8da 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -208,7 +208,7 @@ org.tachyonproject tachyon - 0.4.1 + 0.4.1-thrift org.apache.hadoop diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 774b90ae4d836..c6741fd614dbe 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -188,8 +188,8 @@ object SparkBuild extends Build { "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", // For Sonatype publishing - "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", // also check the local Maven repository ~/.m2 Resolver.mavenLocal ), @@ -322,7 +322,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), + "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro From fd84156d9baabc4bbba446248067a7d771e7ede6 Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 27 Mar 2014 22:33:18 +0800 Subject: [PATCH 26/50] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix --- .../apache/spark/network/netty/TachyonFilePathResolver.java | 2 +- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java b/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java index 483de58c4976c..49c550fff6e3a 100644 --- a/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java +++ b/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java @@ -21,6 +21,6 @@ import org.apache.spark.storage.TachyonFileSegment; public interface TachyonFilePathResolver { - /** Get the file segment in which the given block resides. */ + /** Get the file segment in which the given block resides. This is not a user-facing API*/ TachyonFileSegment getBlockLocation(BlockId blockId); } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c3f3b1ea56b36..7f763163dabb4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -21,6 +21,7 @@ import java.io._ import java.net.URI import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger +import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -42,7 +43,6 @@ import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} -import java.util.Random /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -128,7 +128,7 @@ class SparkContext( // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe - val tachyonFolderName = new Random().nextInt() + "_" + System.currentTimeMillis() + val tachyonFolderName = "spark-" + randomUUID.toString() conf.set("spark.tachyonstore.foldername", tachyonFolderName) val isLocal = (master == "local" || master.startsWith("local[")) 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 e4b6142c223f1..aae061d4d93a2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -58,7 +58,7 @@ private[spark] class BlockManager( private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) var tachyonInitialized = false - private[storage] lazy val tachyonStore : TachyonStore = { + private[storage] lazy val tachyonStore: TachyonStore = { val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) val appFolderName = conf.get("spark.tachyonstore.foldername") val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" @@ -1000,7 +1000,7 @@ private[spark] class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() - if(tachyonInitialized) { + if (tachyonInitialized) { tachyonStore.clear() } metadataCleaner.cancel() From e700d9c4ef1c88eaa635e6ae33008cf4d6884294 Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 27 Mar 2014 22:44:10 +0800 Subject: [PATCH 27/50] add the SparkTachyonHdfsLR example and some comments --- .../spark/examples/SparkTachyonHdfsLR.scala | 82 +++++++++++++++++++ .../spark/examples/SparkTachyonPi.scala | 4 +- 2 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala new file mode 100644 index 0000000000000..44285b1e2a869 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -0,0 +1,82 @@ +/* + * 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.examples + +import java.util.Random +import scala.math.exp +import org.apache.spark.util.Vector +import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.InputFormatInfo +import org.apache.spark.storage.StorageLevel + +/** + * Logistic regression based classification. + * This example uses Tachyon to persist rdds during computation. + */ +object SparkTachyonHdfsLR { + val D = 10 // Numer of dimensions + val rand = new Random(42) + + case class DataPoint(x: Vector, y: Double) + + def parsePoint(line: String): DataPoint = { + //val nums = line.split(' ').map(_.toDouble) + //return DataPoint(new Vector(nums.slice(1, D+1)), nums(0)) + val tok = new java.util.StringTokenizer(line, " ") + var y = tok.nextToken.toDouble + var x = new Array[Double](D) + var i = 0 + while (i < D) { + x(i) = tok.nextToken.toDouble; i += 1 + } + DataPoint(new Vector(x), y) + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: SparkTachyonHdfsLR ") + System.exit(1) + } + val inputPath = args(1) + val conf = SparkHadoopUtil.get.newConfiguration() + val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + InputFormatInfo.computePreferredLocations( + Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) + )) + val lines = sc.textFile(inputPath) + val points = lines.map(parsePoint _).persist(StorageLevel.TACHYON) + val ITERATIONS = args(2).toInt + + // Initialize w to a random value + var w = Vector(D, _ => 2 * rand.nextDouble - 1) + println("Initial w: " + w) + + for (i <- 1 to ITERATIONS) { + println("On iteration " + i) + val gradient = points.map { p => + (1 / (1 + exp(-p.y * (w dot p.x))) - 1) * p.y * p.x + }.reduce(_ + _) + w -= gradient + } + + println("Final w: " + w) + System.exit(0) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index d5e43ab772ce8..b264d9d8f81db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -22,7 +22,9 @@ import scala.math.random import org.apache.spark._ import org.apache.spark.storage.StorageLevel -/** Computes an approximation to pi */ +/** Computes an approximation to pi + * This example uses Tachyon to persist rdds during computation. + */ object SparkTachyonPi { def main(args: Array[String]) { if (args.length == 0) { From 76805aabc74bd29b9c705d923a7e6085a69dce7a Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 27 Mar 2014 23:34:58 +0800 Subject: [PATCH 28/50] unifies the config properties name prefix; add the configs into docs/configuration.md --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 6 +++--- .../spark/storage/TachyonBlockManager.scala | 6 +++--- docs/configuration.md | 15 +++++++++++++++ 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7f763163dabb4..d277f1ef63027 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -129,7 +129,7 @@ class SparkContext( // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() - conf.set("spark.tachyonstore.foldername", tachyonFolderName) + conf.set("spark.tachyonStore.folderName", tachyonFolderName) val isLocal = (master == "local" || master.startsWith("local[")) 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 aae061d4d93a2..f34a8e7fd82e0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -59,10 +59,10 @@ private[spark] class BlockManager( private[storage] val diskStore = new DiskStore(this, diskBlockManager) var tachyonInitialized = false private[storage] lazy val tachyonStore: TachyonStore = { - val storeDir = conf.get("spark.tachyonstore.dir", System.getProperty("java.io.tmpdir")) - val appFolderName = conf.get("spark.tachyonstore.foldername") + val storeDir = conf.get("spark.tachyonStore.baseDir", System.getProperty("java.io.tmpdir")) + val appFolderName = conf.get("spark.tachyonStore.folderName") val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" - val tachyonMaster = conf.get("spark.tachyonmaster.address", "tachyon://localhost:19998") + val tachyonMaster = conf.get("spark.tachyonStore.URL", "tachyon://localhost:19998") val tachyonBlockManager = new TachyonBlockManager( shuffleBlockManager, tachyonStorePath, tachyonMaster) tachyonInitialized = true diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 026de9f028a78..bf526a03c625b 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -52,9 +52,9 @@ private[spark] class TachyonBlockManager( private val subDirsPerTachyonDir = shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt - // Create one Tachyon directory for each path mentioned in spark.tachyon.dir; then, inside this - // directory, create multiple subdirectories that we will hash files into, in order to avoid - // having really large inodes at the top level in Tachyon. + // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName.dir; + // then, inside this directory, create multiple subdirectories that we will hash files into, + // in order to avoid having really large inodes at the top level in Tachyon. private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) diff --git a/docs/configuration.md b/docs/configuration.md index 16ee5ec0f230f..908732392fff1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -122,6 +122,21 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + + + + + + + + + From c9aeabffdd9203baa806cc913cb3a5d4576ac3ba Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 28 Mar 2014 00:26:13 +0800 Subject: [PATCH 29/50] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP --- .../scala/org/apache/spark/storage/StorageLevel.scala | 2 +- .../org/apache/spark/storage/BlockManagerSuite.scala | 8 ++++---- .../org/apache/spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../scala/org/apache/spark/examples/SparkTachyonPi.scala | 2 +- project/SparkBuild.scala | 4 ++-- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 414f5d03c261f..1d8d88bbcb94c 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -133,7 +133,7 @@ object StorageLevel { val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) - val TACHYON = new StorageLevel(false, false, true, false) + val OFF_HEAP = new StorageLevel(false, false, true, false) /** Create a new StorageLevel object */ def apply(useDisk: Boolean, useMemory: Boolean, useTachyon: Boolean, diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d18cfb7d0dd01..909d0cead4cf1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -417,11 +417,11 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.TACHYON) - store.putSingle("a2", a2, StorageLevel.TACHYON) - store.putSingle("a3", a3, StorageLevel.TACHYON) - assert(store.getSingle("a2").isDefined, "a2 was in store") + store.putSingle("a1", a1, StorageLevel.OFF_HEAP) + store.putSingle("a2", a2, StorageLevel.OFF_HEAP) + store.putSingle("a3", a3, StorageLevel.OFF_HEAP) assert(store.getSingle("a3").isDefined, "a3 was in store") + assert(store.getSingle("a2").isDefined, "a2 was in store") assert(store.getSingle("a1").isDefined, "a1 was in store") } else { info("tachyon storage test disabled.") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 44285b1e2a869..4e52e829703ee 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -61,7 +61,7 @@ object SparkTachyonHdfsLR { Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) val lines = sc.textFile(inputPath) - val points = lines.map(parsePoint _).persist(StorageLevel.TACHYON) + val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) val ITERATIONS = args(2).toInt // Initialize w to a random value diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index b264d9d8f81db..8d2ee3c50a749 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -47,7 +47,7 @@ object SparkTachyonPi { println("1- Pi is roughly " + 4.0 * count / n) val rdd2 = spark.parallelize(1 to n, slices) - rdd2.persist(StorageLevel.TACHYON) + rdd2.persist(StorageLevel.OFF_HEAP) val count2 = rdd2.map { i => val x = random * 2 - 1 val y = random * 2 - 1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c6741fd614dbe..a8ca192b4a9fe 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -182,11 +182,11 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( - "Maven Repository" at "https://repo.maven.apache.org/maven2", + "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/", // For Sonatype publishing // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", From 04301d33c325bbda9d3d3e5ccff60dba1d25e248 Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 28 Mar 2014 00:51:56 +0800 Subject: [PATCH 30/50] rename StorageLevel.TACHYON to Storage.OFF_HEAP --- core/src/main/java/org/apache/spark/api/java/StorageLevels.java | 2 +- .../main/scala/org/apache/spark/examples/SparkTachyonPi.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 0944f1089f5a3..c9f5476e4b9b8 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -34,7 +34,7 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); - public static final StorageLevel TACHYON = create(false, false, true, false, 1); + public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1); /** * Create a new StorageLevel object. diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 8d2ee3c50a749..eeae2db973eeb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -38,7 +38,7 @@ object SparkTachyonPi { val n = 100000 * slices val rdd = spark.parallelize(1 to n, slices) - rdd.persist(StorageLevel.TACHYON) + rdd.persist(StorageLevel.OFF_HEAP) val count = rdd.map { i => val x = random * 2 - 1 val y = random * 2 - 1 From 4572f9f61c6b2fc1300bc5efb0c0c59fc19353e0 Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 28 Mar 2014 01:43:03 +0800 Subject: [PATCH 31/50] reserving the old apply function API of StorageLevel --- .../scala/org/apache/spark/storage/StorageLevel.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1d8d88bbcb94c..68ef36b743dd6 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -135,10 +135,15 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object */ + /** Create a new StorageLevel object without setting useTachyon*/ def apply(useDisk: Boolean, useMemory: Boolean, useTachyon: Boolean, - deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( + deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication)) + + /** Create a new StorageLevel object */ + def apply(useDisk: Boolean, useMemory: Boolean, + deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, false, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int) = From 49cc72464c68337fd14a911b2d1341e5bdc32ff4 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 28 Mar 2014 11:58:32 -0700 Subject: [PATCH 32/50] update docs with off_headp option --- docs/scala-programming-guide.md | 40 ++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 13 deletions(-) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 62d6ffc7df6c9..e966a3466175b 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -278,7 +278,8 @@ that originally created it. In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), -or even replicate it across nodes. These levels are chosen by passing a +or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). +These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of @@ -308,6 +309,14 @@ available storage levels is: + + + + @@ -324,23 +333,28 @@ Spark's storage levels are meant to provide different trade-offs between memory efficiency. It allows uses to choose memory, disk, or Tachyon for storing data. We recommend going through the following process to select one: -* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most - CPU-efficient option, allowing operations on the RDDs to run as fast as possible. -* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects - much more space-efficient, but still reasonably fast to access. You can also use `Tachyon` mode - to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will significantly - reduce JVM GC overhead. -* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large - amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk. -* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web - application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones - let you continue running tasks on the RDD without waiting to recompute a lost partition. +* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. + This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. + +* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to +make the objects much more space-efficient, but still reasonably fast to access. You can also use +`OFF_HEAP` mode to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will +significantly reduce JVM GC overhead. + +* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter +a large amount of the data. Otherwise, recomputing a partition is about as fast as reading it from +disk. + +* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve +requests from a web application). *All* the storage levels provide full fault tolerance by +recomputing lost data, but the replicated ones let you continue running tasks on the RDD without +waiting to recompute a lost partition. If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. -Spark has a block manager inside the Executors that let you chose memory, disk, or Tachyon. The +Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system [Tachyon](http://tachyon-project.org/). This mode has the following advantages: From be79d775c36b8c4275de28b9315b804c184ad3d6 Mon Sep 17 00:00:00 2001 From: RongGu Date: Sat, 29 Mar 2014 09:46:15 +0800 Subject: [PATCH 33/50] find a way to clean up some unnecessay metods and classed to make the code simpler --- .../netty/TachyonFilePathResolver.java | 26 ---------------- .../spark/storage/TachyonBlockManager.scala | 16 ++-------- .../apache/spark/storage/TachyonStore.scala | 31 ++++++------------- .../scala/org/apache/spark/util/Utils.scala | 5 +-- 4 files changed, 15 insertions(+), 63 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java diff --git a/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java b/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java deleted file mode 100644 index 49c550fff6e3a..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/TachyonFilePathResolver.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.netty; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.TachyonFileSegment; - -public interface TachyonFilePathResolver { - /** Get the file segment in which the given block resides. This is not a user-facing API*/ - TachyonFileSegment getBlockLocation(BlockId blockId); -} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index bf526a03c625b..4577b2db0d5cf 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,7 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.{TachyonFilePathResolver, ShuffleSender} +import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -39,7 +39,7 @@ private[spark] class TachyonBlockManager( shuffleManager: ShuffleBlockManager, rootDirs: String, val master: String) - extends TachyonFilePathResolver with Logging { + extends Logging { val client = if (master != null && master != "") TachyonFS.get(master) else null @@ -50,7 +50,7 @@ private[spark] class TachyonBlockManager( private val MAX_DIR_CREATION_ATTEMPTS = 10 private val subDirsPerTachyonDir = - shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt + shuffleManager.conf.get("spark.tachyonStore.subDirectories", "4").toInt // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName.dir; // then, inside this directory, create multiple subdirectories that we will hash files into, @@ -59,16 +59,6 @@ private[spark] class TachyonBlockManager( private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) addShutdownHook() - - /** - * Returns the physical tachyon file segment in which the given BlockId is located. - * If the BlockId has been mapped to a specific FileSegment, that will be returned. - * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly. - */ - def getBlockLocation(blockId: BlockId): TachyonFileSegment = { - val file = getFile(blockId.name) - new TachyonFileSegment(file, 0, file.length()) - } def removeFile(file: TachyonFile): Boolean = { client.delete(file.getPath(), false) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 7248a4231bc7b..ebfa69f4f5ff5 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -21,8 +21,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import tachyon.client.WriteType -import tachyon.client.ReadType +import tachyon.client.{WriteType, ReadType} import org.apache.spark.Logging import org.apache.spark.util.Utils @@ -43,7 +42,8 @@ private class TachyonStore( logInfo("TachyonStore started") override def getSize(blockId: BlockId): Long = { - tachyonManager.getBlockLocation(blockId).length + tachyonManager.getFile(blockId.name).length +// tachyonManager.getBlockLocation(blockId).length } override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { @@ -94,15 +94,11 @@ private class TachyonStore( } override def remove(blockId: BlockId): Boolean = { - val fileSegment = tachyonManager.getBlockLocation(blockId) - val file = fileSegment.file - if (tachyonManager.fileExists(file) && file.length() == fileSegment.length) { + val file = tachyonManager.getFile(blockId) + if (tachyonManager.fileExists(file)) { tachyonManager.removeFile(file) } else { - if (fileSegment.length < file.length()) { - logWarning("Could not delete block associated with only a part of a file: " + blockId) - } - false + true } } @@ -110,31 +106,22 @@ private class TachyonStore( getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) } - /** - * A version of getValues that allows a custom serializer. This is used as part of the - * shuffle short-circuit code. - */ - def getValues(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer)) - } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val segment = tachyonManager.getBlockLocation(blockId) val file = tachyonManager.getFile(blockId) val is = file.getInStream(ReadType.CACHE) var buffer: ByteBuffer = null if (is != null){ - val size = segment.length - segment.offset + val size = file.length val bs = new Array[Byte](size.asInstanceOf[Int]) - is.read(bs, segment.offset.asInstanceOf[Int] , size.asInstanceOf[Int]) + is.read(bs, 0, size.asInstanceOf[Int]) buffer = ByteBuffer.wrap(bs) } Some(buffer) } override def contains(blockId: BlockId): Boolean = { - val fileSegment = tachyonManager.getBlockLocation(blockId) - val file = fileSegment.file + val file = tachyonManager.getFile(blockId) tachyonManager.fileExists(file) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0130254f55a59..dad0d5156112f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -33,13 +33,14 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ +import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import tachyon.client.TachyonFile -import tachyon.client.TachyonFS + + /** From 619a9a82f78a0dae1dedec38629810c150f9180c Mon Sep 17 00:00:00 2001 From: RongGu Date: Sat, 29 Mar 2014 10:48:33 +0800 Subject: [PATCH 34/50] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore --- .../scala/org/apache/spark/storage/TachyonBlockManager.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 4577b2db0d5cf..0b4082de04162 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -50,7 +50,7 @@ private[spark] class TachyonBlockManager( private val MAX_DIR_CREATION_ATTEMPTS = 10 private val subDirsPerTachyonDir = - shuffleManager.conf.get("spark.tachyonStore.subDirectories", "4").toInt + shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName.dir; // then, inside this directory, create multiple subdirectories that we will hash files into, @@ -100,6 +100,7 @@ private[spark] class TachyonBlockManager( def getFile(blockId: BlockId): TachyonFile = getFile(blockId.name) + // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. private def createTachyonDirs(): Array[TachyonFile] = { logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") From 64348b2d580c19622a617f7fe57b599f92da7ba2 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 28 Mar 2014 12:03:24 -0700 Subject: [PATCH 35/50] update conf docs. --- docs/configuration.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 908732392fff1..7c8eea0ba012a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -124,9 +124,9 @@ Apart from these, the following properties are also available, and may be useful - + @@ -176,13 +176,13 @@ Apart from these, the following properties are also available, and may be useful - + @@ -390,7 +390,7 @@ Apart from these, the following properties are also available, and may be useful @@ -445,7 +445,7 @@ Apart from these, the following properties are also available, and may be useful @@ -570,7 +570,7 @@ Apart from these, the following properties are also available, and may be useful the driver. - + - + - +
Storage LevelMeaning
MEMORY_AND_DISK_SER Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them - on the fly each time they're needed. Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of + recomputing them on the fly each time they're needed.
DISK_ONLY
spark.tachyonStore.baseDir/tmp/ + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.URL. + It can also be a comma-separated list of multiple directories on Tachyon file system. +
spark.tachyonStore.URLtachyon://localhost:19998 + The URL of the underlying Tachyon file system in the TachyonStore. +
spark.mesos.coarse false Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them on the fly each time they're needed.
OFF_HEAP Store RDD in a serialized format in Tachyon. + This is generally more space-efficient than deserialized objects, especially when using a + fast serializer, but more CPU-intensive to read. + This also significantly reduces the overheads of GC. +
DISK_ONLY Store the RDD partitions only on disk.
spark.tachyonStore.baseDir/tmp/System.getProperty("java.io.tmpdir") - Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.URL. + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.URL. It can also be a comma-separated list of multiple directories on Tachyon file system.
spark.ui.acls.enable false - Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has + Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has access permissions to view the web ui. See spark.ui.view.acls for more details. Also note this requires the user to be known, if the user comes across as null no checks are done. Filters can be used to authenticate and set the user.
spark.ui.view.acls Empty @@ -291,10 +291,10 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset 10000 - When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches - objects to prevent writing redundant data, however that stops garbage collection of those - objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value of <= 0. + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to a value of <= 0. By default it will reset the serializer every 10,000 objects.
spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
spark.broadcast.blockSize 4096 - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Size of each piece of a block in kilobytes for TorrentBroadcastFactory. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit.
spark.authenticate false @@ -578,7 +578,7 @@ Apart from these, the following properties are also available, and may be useful running on Yarn.
spark.authenticate.secret None @@ -586,12 +586,12 @@ Apart from these, the following properties are also available, and may be useful not running on Yarn and authentication is enabled.
spark.core.connection.auth.wait.timeout 30 Number of seconds for the connection to wait for authentication to occur before timing - out and giving up. + out and giving up.
From 589eafeec21c0a40af6f6a64a63019db30b43b4f Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 28 Mar 2014 12:56:46 -0700 Subject: [PATCH 36/50] use TRY_CACHE instead of MUST_CACHE --- .../org/apache/spark/storage/TachyonStore.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index ebfa69f4f5ff5..7dec3f25c192a 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -38,7 +38,7 @@ private class TachyonStore( blockManager: BlockManager, tachyonManager: TachyonBlockManager) extends BlockStore(blockManager: BlockManager) with Logging { - + logInfo("TachyonStore started") override def getSize(blockId: BlockId): Long = { @@ -49,7 +49,7 @@ private class TachyonStore( override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { putToTachyonStore(blockId, _bytes, true) } - + override def putValues( blockId: BlockId, values: ArrayBuffer[Any], @@ -79,13 +79,13 @@ private class TachyonStore( logDebug("Attempting to put block " + blockId + " into Tachyon") val startTime = System.currentTimeMillis val file = tachyonManager.getFile(blockId) - val os = file.getOutStream(WriteType.MUST_CACHE) + val os = file.getOutStream(WriteType.TRY_CACHE) os.write(bytes.array()) os.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file in Tachyon in %d ms".format( blockId, Utils.bytesToString(bytes.limit), (finishTime - startTime))) - + if (returnValues) { PutResult(_bytes.limit(), Right(_bytes.duplicate())) } else { @@ -105,7 +105,7 @@ private class TachyonStore( override def getValues(blockId: BlockId): Option[Iterator[Any]] = { getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) } - + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val file = tachyonManager.getFile(blockId) @@ -115,8 +115,8 @@ private class TachyonStore( val size = file.length val bs = new Array[Byte](size.asInstanceOf[Int]) is.read(bs, 0, size.asInstanceOf[Int]) - buffer = ByteBuffer.wrap(bs) - } + buffer = ByteBuffer.wrap(bs) + } Some(buffer) } From 91fa09d75fc853f6a915299ae740e79fc849ba56 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 28 Mar 2014 14:14:55 -0700 Subject: [PATCH 37/50] address patrick's comments --- .../apache/spark/storage/TachyonStore.scala | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 7dec3f25c192a..ec6759aa8cd0b 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -17,6 +17,7 @@ package org.apache.spark.storage +import java.io.IOException import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer @@ -109,13 +110,28 @@ private class TachyonStore( override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val file = tachyonManager.getFile(blockId) + if (file == null || file.getLocationHosts().size == 0) { + return None + } val is = file.getInStream(ReadType.CACHE) var buffer: ByteBuffer = null - if (is != null){ - val size = file.length - val bs = new Array[Byte](size.asInstanceOf[Int]) - is.read(bs, 0, size.asInstanceOf[Int]) - buffer = ByteBuffer.wrap(bs) + try { + if (is != null) { + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) + buffer = ByteBuffer.wrap(bs) + if (fetchSize != size) { + logWarning("Failed to fetch the block " + blockId + " from Tachyon : Size " + size + + " is not equal to fetched size " + fetchSize) + return None + } + } + } catch { + case ioe: IOException => { + logWarning("Failed to fetch the block " + blockId + " from Tachyon", ioe) + return None + } } Some(buffer) } From 3dcace4b2be6f11a359faa6f08a25561736b64b1 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Tue, 1 Apr 2014 23:34:18 -0700 Subject: [PATCH 38/50] address matei's comments --- .../spark/executor/ExecutorExitCode.scala | 5 +++- .../apache/spark/storage/BlockManager.scala | 12 +++++----- .../storage/BlockManagerMasterActor.scala | 4 ++-- .../apache/spark/storage/StorageLevel.scala | 24 +++++++++---------- .../org/apache/spark/util/JsonProtocol.scala | 2 +- docs/configuration.md | 4 ++-- .../spark/examples/SparkTachyonPi.scala | 14 +++-------- python/pyspark/storagelevel.py | 4 ++-- 8 files changed, 32 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index d9bad10507ffa..ceff3a067d72a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -41,7 +41,7 @@ object ExecutorExitCode { /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 - /** TachyonStore failed to create a local temporary directory after many attempts. */ + /** TachyonStore failed to initialize after many attempts. */ val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 /** TachyonStore failed to create a local temporary directory after many attempts. */ @@ -54,6 +54,9 @@ object ExecutorExitCode { case OOM => "OutOfMemoryError" case DISK_STORE_FAILED_TO_CREATE_DIR => "Failed to create local directory (bad spark.local.dir?)" + case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." + case TACHYON_STORE_FAILED_TO_CREATE_DIR => + "TachyonStore failed to create a local temporary directory." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { 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 f34a8e7fd82e0..4bd0563fc846d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -62,7 +62,7 @@ private[spark] class BlockManager( val storeDir = conf.get("spark.tachyonStore.baseDir", System.getProperty("java.io.tmpdir")) val appFolderName = conf.get("spark.tachyonStore.folderName") val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" - val tachyonMaster = conf.get("spark.tachyonStore.URL", "tachyon://localhost:19998") + val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") val tachyonBlockManager = new TachyonBlockManager( shuffleBlockManager, tachyonStorePath, tachyonMaster) tachyonInitialized = true @@ -276,7 +276,7 @@ private[spark] class BlockManager( (StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) - val inTachyon = level.useTachyon && tachyonStore.contains(blockId) + val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false val replication = if (inMem || inTachyon || onDisk) level.replication else 1 @@ -369,7 +369,7 @@ private[spark] class BlockManager( } // Look for the block in Tachyon - if (level.useTachyon) { + if (level.useOffHeap) { logDebug("Getting block " + blockId + " from tachyon") if (tachyonStore.contains(blockId)) { tachyonStore.getBytes(blockId) match { @@ -651,7 +651,7 @@ private[spark] class BlockManager( } // Keep track of which blocks are dropped from memory res.droppedBlocks.foreach { block => updatedBlocks += block } - } else if (level.useTachyon) { + } else if (level.useOffHeap) { // Save to Tachyon. val askForBytes = level.replication > 1 val res = data match { @@ -757,7 +757,7 @@ private[spark] class BlockManager( var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { val tLevel = StorageLevel( - level.useDisk, level.useMemory, level.useTachyon, level.deserialized, 1) + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) if (cachedPeers == null) { cachedPeers = master.getPeers(blockManagerId, level.replication - 1) } @@ -922,7 +922,7 @@ private[spark] class BlockManager( if (level.useDisk) { diskStore.remove(id) } - if (level.useTachyon) { + if (level.useOffHeap) { tachyonStore.remove(id) } iterator.remove() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 4f2e313827835..ca2eb02c5f4e7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -375,7 +375,7 @@ private[spark] class BlockManagerInfo( logInfo("Added %s on disk on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } - if (storageLevel.useTachyon) { + if (storageLevel.useOffHeap) { _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, tachyonSize)) logInfo("Added %s on tachyon on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(tachyonSize))) @@ -394,7 +394,7 @@ private[spark] class BlockManagerInfo( logInfo("Removed %s on %s on disk (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } - if (blockStatus.storageLevel.useTachyon) { + if (blockStatus.storageLevel.useOffHeap) { logInfo("Removed %s on %s on tachyon (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize))) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 68ef36b743dd6..9cd4a42e2ad6b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -31,7 +31,7 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, - private var useTachyon_ : Boolean, + private var useOffHeap_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) extends Externalizable { @@ -45,27 +45,27 @@ class StorageLevel private( def useDisk = useDisk_ def useMemory = useMemory_ - def useTachyon = useTachyon_ + def useOffHeap = useOffHeap_ def deserialized = deserialized_ def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.useTachyon, this.deserialized, this.replication) + this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) override def equals(other: Any): Boolean = other match { case s: StorageLevel => s.useDisk == useDisk && s.useMemory == useMemory && - s.useTachyon == useTachyon && + s.useOffHeap == useOffHeap && s.deserialized == deserialized && s.replication == replication case _ => false } - def isValid = ((useMemory || useDisk || useTachyon) && (replication > 0)) + def isValid = ((useMemory || useDisk || useOffHeap) && (replication > 0)) def toInt: Int = { var ret = 0 @@ -75,7 +75,7 @@ class StorageLevel private( if (useMemory_) { ret |= 4 } - if (useTachyon_) { + if (useOffHeap_) { ret |= 2 } if (deserialized_) { @@ -93,7 +93,7 @@ class StorageLevel private( val flags = in.readByte() useDisk_ = (flags & 8) != 0 useMemory_ = (flags & 4) != 0 - useTachyon_ = (flags & 2) != 0 + useOffHeap_ = (flags & 2) != 0 deserialized_ = (flags & 1) != 0 replication_ = in.readByte() } @@ -102,14 +102,14 @@ class StorageLevel private( private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( - useDisk, useMemory, useTachyon, deserialized, replication) + useDisk, useMemory, useOffHeap, deserialized, replication) override def hashCode(): Int = toInt * 41 + replication def description : String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") - result += (if (useTachyon) "Tachyon " else "") + result += (if (useOffHeap) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") result += "%sx Replicated".format(replication) result @@ -135,10 +135,10 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useTachyon*/ - def apply(useDisk: Boolean, useMemory: Boolean, useTachyon: Boolean, + /** Create a new StorageLevel object without setting useOffHeap*/ + def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( - new StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication)) + new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) /** Create a new StorageLevel object */ def apply(useDisk: Boolean, useMemory: Boolean, diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 99cbed29bb18a..331cffc8d53c9 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -281,7 +281,7 @@ private[spark] object JsonProtocol { def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ - ("Use Tachyon" -> storageLevel.useTachyon) ~ + ("Use Tachyon" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } diff --git a/docs/configuration.md b/docs/configuration.md index 7c8eea0ba012a..29b8b11539244 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -126,12 +126,12 @@ Apart from these, the following properties are also available, and may be useful spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") - Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.URL. + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url. It can also be a comma-separated list of multiple directories on Tachyon file system. - spark.tachyonStore.URL + spark.tachyonStore.url tachyon://localhost:19998 The URL of the underlying Tachyon file system in the TachyonStore. diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index eeae2db973eeb..ce78f0876ed7c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -22,7 +22,8 @@ import scala.math.random import org.apache.spark._ import org.apache.spark.storage.StorageLevel -/** Computes an approximation to pi +/** + * Computes an approximation to pi * This example uses Tachyon to persist rdds during computation. */ object SparkTachyonPi { @@ -44,16 +45,7 @@ object SparkTachyonPi { val y = random * 2 - 1 if (x * x + y * y < 1) 1 else 0 }.reduce(_ + _) - println("1- Pi is roughly " + 4.0 * count / n) - - val rdd2 = spark.parallelize(1 to n, slices) - rdd2.persist(StorageLevel.OFF_HEAP) - val count2 = rdd2.map { i => - val x = random * 2 - 1 - val y = random * 2 - 1 - if (x * x + y * y < 1) 1 else 0 - }.reduce(_ + _) - println("2- Pi is roughly " + 4.0 * count2 / n) + println("Pi is roughly " + 4.0 * count / n) spark.stop() } diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 1f5e877f9e589..ebc813b87f4d5 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -34,7 +34,7 @@ def __init__(self, useDisk, useMemory, useTachyon, deserialized, replication = 1 def __repr__(self): return "StorageLevel(%s, %s, %s, %s, %s)" % ( - self.useDisk, self.useMemory, self.useTachyon, self.deserialized, self.replication) + self.useDisk, self.useMemory, self.useOffHeap, self.deserialized, self.replication) StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False) StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2) @@ -46,4 +46,4 @@ def __repr__(self): StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) -StorageLevel.TACHYON = StorageLevel(False, False, True, False, 1) \ No newline at end of file +StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) \ No newline at end of file From 77d27037463ea1ece927a4e913863283588f0ef6 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Tue, 1 Apr 2014 23:38:27 -0700 Subject: [PATCH 39/50] change python api.git status --- .../main/java/org/apache/spark/api/java/StorageLevels.java | 6 +++--- python/pyspark/storagelevel.py | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index c9f5476e4b9b8..840a1bd93bfbb 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -53,16 +53,16 @@ public static StorageLevel create(boolean useDisk, boolean useMemory, boolean de * Create a new StorageLevel object. * @param useDisk saved to disk, if true * @param useMemory saved to memory, if true - * @param useTachyon saved to Tachyon, if true + * @param useOffHeap saved to Tachyon, if true * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ public static StorageLevel create( boolean useDisk, boolean useMemory, - boolean useTachyon, + boolean useOffHeap, boolean deserialized, int replication) { - return StorageLevel.apply(useDisk, useMemory, useTachyon, deserialized, replication); + return StorageLevel.apply(useDisk, useMemory, useOffHeap, deserialized, replication); } } diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index ebc813b87f4d5..7b6660eab231b 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -25,10 +25,10 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, useTachyon, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1): self.useDisk = useDisk self.useMemory = useMemory - self.useTachyon = useTachyon + self.useOffHeap = useOffHeap self.deserialized = deserialized self.replication = replication From d9a6438007d3c8cad0a0a552dc6349008dde61e1 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Tue, 1 Apr 2014 23:41:41 -0700 Subject: [PATCH 40/50] fix for pspark --- python/pyspark/context.py | 7 +++++-- python/pyspark/rdd.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5c98db235f3f8..a97788a61240e 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -383,8 +383,11 @@ def _getJavaStorageLevel(self, storageLevel): raise Exception("storageLevel must be of type pyspark.StorageLevel") newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel - return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.useTachyon, - storageLevel.deserialized, storageLevel.replication) + return newStorageLevel(storageLevel.useDisk, + storageLevel.useMemory, + storageLevel.useOffHeap, + storageLevel.deserialized, + storageLevel.replication) def setJobGroup(self, groupId, description): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3269e5613aac9..f49032c7949e3 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1210,7 +1210,7 @@ def getStorageLevel(self): java_storage_level = self._jrdd.getStorageLevel() storage_level = StorageLevel(java_storage_level.useDisk(), java_storage_level.useMemory(), - java_storage_level.useTachyon(), + java_storage_level.useOffHeap(), java_storage_level.deserialized(), java_storage_level.replication()) return storage_level From 9b979354b24cceb0577a013e3400222553b6d4df Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Wed, 2 Apr 2014 00:22:45 -0700 Subject: [PATCH 41/50] address aaron's comments --- .../main/scala/org/apache/spark/SparkContext.scala | 4 ++-- .../apache/spark/storage/TachyonBlockManager.scala | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d277f1ef63027..b1d3c8ad9b88c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,8 +19,8 @@ package org.apache.spark import java.io._ import java.net.URI -import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger +import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable @@ -125,7 +125,7 @@ class SparkContext( val master = conf.get("spark.master") val appName = conf.get("spark.app.name") - + // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 0b4082de04162..b0b9674856568 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -36,8 +36,8 @@ import org.apache.spark.util.Utils * @param rootDirs The directories to use for storing block files. Data will be hashed among these. */ private[spark] class TachyonBlockManager( - shuffleManager: ShuffleBlockManager, - rootDirs: String, + shuffleManager: ShuffleBlockManager, + rootDirs: String, val master: String) extends Logging { @@ -49,21 +49,21 @@ private[spark] class TachyonBlockManager( } private val MAX_DIR_CREATION_ATTEMPTS = 10 - private val subDirsPerTachyonDir = + private val subDirsPerTachyonDir = shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt - // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName.dir; - // then, inside this directory, create multiple subdirectories that we will hash files into, + // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; + // then, inside this directory, create multiple subdirectories that we will hash files into, // in order to avoid having really large inodes at the top level in Tachyon. private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) addShutdownHook() - + def removeFile(file: TachyonFile): Boolean = { client.delete(file.getPath(), false) } - + def fileExists(file: TachyonFile): Boolean = { client.exist(file.getPath()) } From 5cc041c1677f29c7a93e4149f5c25eb1566fac70 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Wed, 2 Apr 2014 13:59:24 -0700 Subject: [PATCH 42/50] address aaron's comments --- .../apache/spark/storage/TachyonStore.scala | 39 +++++++++---------- project/SparkBuild.scala | 2 - 2 files changed, 19 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index ec6759aa8cd0b..da59d25adb6cf 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -44,53 +44,52 @@ private class TachyonStore( override def getSize(blockId: BlockId): Long = { tachyonManager.getFile(blockId.name).length -// tachyonManager.getBlockLocation(blockId).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { - putToTachyonStore(blockId, _bytes, true) + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putToTachyonStore(blockId, bytes, true) } override def putValues( - blockId: BlockId, - values: ArrayBuffer[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { return putValues(blockId, values.toIterator, level, returnValues) } override def putValues( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { logDebug("Attempting to write values for block " + blockId) val _bytes = blockManager.dataSerialize(blockId, values) putToTachyonStore(blockId, _bytes, returnValues) } private def putToTachyonStore( - blockId: BlockId, - _bytes: ByteBuffer, - returnValues: Boolean): PutResult = { + blockId: BlockId, + bytes: ByteBuffer, + returnValues: Boolean): PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive - val bytes = _bytes.duplicate() - bytes.rewind() + val byteBuffer = bytes.duplicate() + byteBuffer.rewind() logDebug("Attempting to put block " + blockId + " into Tachyon") val startTime = System.currentTimeMillis val file = tachyonManager.getFile(blockId) val os = file.getOutStream(WriteType.TRY_CACHE) - os.write(bytes.array()) + os.write(byteBuffer.array()) os.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file in Tachyon in %d ms".format( - blockId, Utils.bytesToString(bytes.limit), (finishTime - startTime))) + blockId, Utils.bytesToString(byteBuffer.limit), (finishTime - startTime))) if (returnValues) { - PutResult(_bytes.limit(), Right(_bytes.duplicate())) + PutResult(bytes.limit(), Right(bytes.duplicate())) } else { - PutResult(_bytes.limit(), null) + PutResult(bytes.limit(), null) } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a8ca192b4a9fe..2beafd880dd63 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -283,8 +283,6 @@ object SparkBuild extends Build { val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") - val excludeKyro = ExclusionRule(organization = "de.javakaffee") - val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") From 120e48a0bc7b33891d46fbc1daa26b8952edc559 Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 3 Apr 2014 18:31:14 +0800 Subject: [PATCH 43/50] changed the root-level dir name in Tachyon --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4bd0563fc846d..e30df259efc01 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -59,7 +59,7 @@ private[spark] class BlockManager( private[storage] val diskStore = new DiskStore(this, diskBlockManager) var tachyonInitialized = false private[storage] lazy val tachyonStore: TachyonStore = { - val storeDir = conf.get("spark.tachyonStore.baseDir", System.getProperty("java.io.tmpdir")) + val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") val appFolderName = conf.get("spark.tachyonStore.folderName") val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") From 8adfcfa55bdb5c2a0725c4ffa6632ce9442e78af Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 3 Apr 2014 18:52:53 +0800 Subject: [PATCH 44/50] address arron's comment on inTachyonSize --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 e30df259efc01..0abf75539bb8e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -257,7 +257,7 @@ private[spark] class BlockManager( if (info.tellMaster) { val storageLevel = status.storageLevel val inMemSize = Math.max(status.memSize, droppedMemorySize) - val inTachyonSize = Math.max(status.tachyonSize, droppedMemorySize) + val inTachyonSize = status.tachyonSize val onDiskSize = status.diskSize master.updateBlockInfo( blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index ca2eb02c5f4e7..378f4cadc17d7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -357,7 +357,7 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory or on-disk. + /* isValid means it is either stored in-memory, on-disk or on-Tachyon. * But the memSize here indicates the data size in or dropped from memory, * tachyonSize here indicates the data size in or dropped from Tachyon, * and the diskSize here indicates the data size in or dropped to disk. From 51149e71898493e7aa314d0293a4fe1ed6eaefde Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 3 Apr 2014 19:02:02 +0800 Subject: [PATCH 45/50] address aaron's comment on returning value of the remove() function in tachyonstore --- core/src/main/scala/org/apache/spark/storage/TachyonStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index da59d25adb6cf..c37e76f893605 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -98,7 +98,7 @@ private class TachyonStore( if (tachyonManager.fileExists(file)) { tachyonManager.removeFile(file) } else { - true + false } } From 7cd4600a85414e226b6db4263c63ea5380dd2c84 Mon Sep 17 00:00:00 2001 From: RongGu Date: Thu, 3 Apr 2014 19:32:50 +0800 Subject: [PATCH 46/50] remove some logic code for tachyonstore's replication --- .../scala/org/apache/spark/storage/BlockManager.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 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 0abf75539bb8e..9e7e918818623 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -653,12 +653,11 @@ private[spark] class BlockManager( res.droppedBlocks.foreach { block => updatedBlocks += block } } else if (level.useOffHeap) { // Save to Tachyon. - val askForBytes = level.replication > 1 val res = data match { case IteratorValues(iterator) => - tachyonStore.putValues(blockId, iterator, level, askForBytes) + tachyonStore.putValues(blockId, iterator, level, false) case ArrayBufferValues(array) => - tachyonStore.putValues(blockId, array, level, askForBytes) + tachyonStore.putValues(blockId, array, level, false) case ByteBufferValues(bytes) => { bytes.rewind(); tachyonStore.putBytes(blockId, bytes, level) @@ -668,7 +667,7 @@ private[spark] class BlockManager( res.data match { case Right(newBytes) => bytesAfterPut = newBytes case _ => - } + } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -718,7 +717,7 @@ private[spark] class BlockManager( // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: - if (level.replication > 1) { + if (level.replication > 1 && !level.useOffHeap) { data match { case ByteBufferValues(bytes) => Await.ready(replicationFuture, Duration.Inf) case _ => { @@ -740,7 +739,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1) { + if (level.replication > 1 && !level.useOffHeap) { logDebug("Put for block " + blockId + " with replication took " + Utils.getUsedTimeMs(startTimeMs)) } else { From 55b59181bca9488d5323db61a12903a4ee76b8cb Mon Sep 17 00:00:00 2001 From: RongGu Date: Fri, 4 Apr 2014 23:56:54 +0800 Subject: [PATCH 47/50] address matei's comment on the replication of offHeap storagelevel --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- .../main/scala/org/apache/spark/storage/StorageLevel.scala | 3 +++ 2 files changed, 5 insertions(+), 2 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 9e7e918818623..19138d9dde697 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -717,7 +717,7 @@ private[spark] class BlockManager( // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: - if (level.replication > 1 && !level.useOffHeap) { + if (level.replication > 1) { data match { case ByteBufferValues(bytes) => Await.ready(replicationFuture, Duration.Inf) case _ => { @@ -739,7 +739,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1 && !level.useOffHeap) { + if (level.replication > 1) { logDebug("Put for block " + blockId + " with replication took " + Utils.getUsedTimeMs(startTimeMs)) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 9cd4a42e2ad6b..8ceedfdd8640b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -50,6 +50,9 @@ class StorageLevel private( def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + + assert(!(useOffHeap && (replication > 1)), + "The replication of useOffHeap mode can not set more than 1") override def clone(): StorageLevel = new StorageLevel( this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) From e0f489180c6a2316c69c73faf9a9c94915385a00 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 4 Apr 2014 13:27:16 -0700 Subject: [PATCH 48/50] better check offheap. --- .../apache/spark/storage/StorageLevel.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 8ceedfdd8640b..e47355ffc15ed 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,8 +21,8 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, - * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to - * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on + * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to + * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on * multiple nodes. * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the @@ -50,9 +50,13 @@ class StorageLevel private( def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") - - assert(!(useOffHeap && (replication > 1)), - "The replication of useOffHeap mode can not set more than 1") + + if (useOffHeap) { + require(useDisk == false, "Off-heap storage level does not support using disk") + require(useMemory == false, "Off-heap storage level does not support using heap memory") + require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(replication == 1, "Off-heap storage level does not support multiple replication") + } override def clone(): StorageLevel = new StorageLevel( this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) @@ -137,12 +141,12 @@ object StorageLevel { val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - + /** Create a new StorageLevel object without setting useOffHeap*/ def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) - + /** Create a new StorageLevel object */ def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( From ae7834b04d7841e141b6a875c5e300db53bf30af Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 4 Apr 2014 16:12:28 -0700 Subject: [PATCH 49/50] minor cleanup --- core/src/main/scala/org/apache/spark/util/Utils.scala | 11 ++++------- .../org/apache/spark/storage/BlockManagerSuite.scala | 1 + .../apache/spark/examples/SparkTachyonHdfsLR.scala | 6 ++---- 3 files changed, 7 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c9c5dcb730b23..57a9aa09973a5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -40,9 +40,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - - - /** * Various utility methods used by Spark. */ @@ -164,7 +161,7 @@ private[spark] object Utils extends Logging { shutdownDeletePaths += absolutePath } } - + // Register the tachyon path to be deleted via shutdown hook def registerShutdownDeleteDir(tachyonfile: TachyonFile) { val absolutePath = tachyonfile.getPath() @@ -180,7 +177,7 @@ private[spark] object Utils extends Logging { shutdownDeletePaths.contains(absolutePath) } } - + // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() @@ -204,7 +201,7 @@ private[spark] object Utils extends Logging { } retval } - + // Note: if file is child of some registered path, while not equal to it, then return true; // else false. This is to ensure that two shutdown hooks do not try to delete each others // paths - resulting in Exception and incomplete cleanup. @@ -573,7 +570,7 @@ private[spark] object Utils extends Logging { } } } - + /** * Delete a file or directory and its contents recursively. */ diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 909d0cead4cf1..b6dd0526105a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -411,6 +411,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("tachyon storage") { + // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 4e52e829703ee..53b303d658386 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -26,7 +26,7 @@ import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.storage.StorageLevel /** - * Logistic regression based classification. + * Logistic regression based classification. * This example uses Tachyon to persist rdds during computation. */ object SparkTachyonHdfsLR { @@ -36,8 +36,6 @@ object SparkTachyonHdfsLR { case class DataPoint(x: Vector, y: Double) def parsePoint(line: String): DataPoint = { - //val nums = line.split(' ').map(_.toDouble) - //return DataPoint(new Vector(nums.slice(1, D+1)), nums(0)) val tok = new java.util.StringTokenizer(line, " ") var y = tok.nextToken.toDouble var x = new Array[Double](D) @@ -56,7 +54,7 @@ object SparkTachyonHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) From 9f7fa1b20d72b4a6a2b9197e57fd233f85482fc8 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 4 Apr 2014 16:17:56 -0700 Subject: [PATCH 50/50] fix code style --- .../main/scala/org/apache/spark/storage/StorageLevel.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index bc05e0ffff520..95e71de2d3f1d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -142,7 +142,7 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useOffHeap*/ + /** Create a new StorageLevel object without setting useOffHeap */ def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) @@ -150,7 +150,7 @@ object StorageLevel { /** Create a new StorageLevel object */ def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( - new StorageLevel(useDisk, useMemory, false, deserialized, replication)) + new StorageLevel(useDisk, useMemory, false, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int): StorageLevel =