diff --git a/bagel/pom.xml b/bagel/pom.xml index 142f75c5d2c64..355f437c5b16a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -31,20 +31,6 @@ Spark Project Bagel http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a59f599fd7d2..b0218531e9eb8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -28,7 +28,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" . $FWDIR/bin/load-spark-env.sh # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" diff --git a/bin/pyspark b/bin/pyspark index cad982bc33477..f5558853e8a4e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -46,6 +46,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 95791095ec932..d7cfd5eec501c 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -45,6 +45,7 @@ rem Figure out which Python to use. if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% +set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py diff --git a/bin/spark-class b/bin/spark-class index 6871e180c9fa8..e8160c8af64c1 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -73,11 +73,13 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" + # Spark submit uses SPARK_SUBMIT_OPTS and SPARK_JAVA_OPTS + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS \ + -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} ;; + *) OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} @@ -98,7 +100,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$_SPARK_LIBRARY_PATH" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then diff --git a/bin/spark-shell b/bin/spark-shell index f1f3c18877ed4..7f03349c5e910 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -20,7 +20,6 @@ # # Shell script for starting the Spark Shell REPL -args="$@" cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -46,12 +45,12 @@ function main(){ # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main fi } @@ -83,7 +82,7 @@ if [[ ! $? ]]; then saved_stty="" fi -main +main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. diff --git a/bin/spark-submit b/bin/spark-submit index b2a1dca721dff..dd0d95d9d4002 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -26,11 +26,11 @@ while (($#)); do elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 elif [ "$1" = "--driver-library-path" ]; then - export _SPARK_LIBRARY_PATH=$2 + export SPARK_SUBMIT_LIBRARY_PATH=$2 elif [ "$1" = "--driver-class-path" ]; then - export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" + export SPARK_SUBMIT_CLASSPATH=$2 elif [ "$1" = "--driver-java-options" ]; then - export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" + export SPARK_SUBMIT_OPTS=$2 fi shift done diff --git a/core/pom.xml b/core/pom.xml index 058b7acba73ca..822b5b1dd7cc2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -30,19 +30,6 @@ jar Spark Project Core http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.hadoop @@ -147,15 +134,6 @@ org.json4s json4s-jackson_${scala.binary.version} 3.2.6 - - - - org.scala-lang - scalap - - colt @@ -316,6 +294,48 @@ + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + generate-resources + + exec + + + + + unzip + ../python + + -o + lib/py4j*.zip + -d + build + + + + + + + src/main/resources + + + ../python + + pyspark/*.py + + + + ../python/build + + py4j/*.py + + + diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 574a98636a619..af06d1dca94f4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -18,7 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList, Iterator => JIterator} -import java.lang.{Iterable => JIterable} +import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -264,6 +264,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } + /** + * Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, + * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method + * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. + */ + def zipWithUniqueId(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, JLong]] + } + + /** + * Zips this RDD with its element indices. The ordering is first based on the partition index + * and then the ordering of items within each partition. So the first item in the first + * partition gets index 0, and the last item in the last partition receives the largest index. + * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. + * This method needs to trigger a spark job when this RDD contains more than one partitions. + */ + def zipWithIndex(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, JLong]] + } + // Actions (launch a job to return a value to the user program) /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a5f0f3d5e7eae..02799ce0091b0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -78,12 +78,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") - workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() // Redirect the worker's stderr to ours @@ -154,12 +151,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") - workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() // Redirect the stderr to ours diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 24edc60684376..d131f1809c99e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -23,6 +23,7 @@ import java.net.{URI, URL} import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.spark.executor.ExecutorURLClassLoader +import org.apache.spark.util.Utils /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -128,6 +129,18 @@ object SparkSubmit { childArgs += ("--class", appArgs.mainClass) } + if (clusterManager == YARN) { + // The choice of class is arbitrary, could use any spark-yarn class + if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + val msg = "Could not load YARN classes. This copy of Spark may not have been compiled " + + "with YARN support." + throw new Exception(msg) + } + } + + // Special flag to avoid deprecation warnings at the client + sysProps("SPARK_SUBMIT") = "true" + val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 58d9e9add764a..5834dc40f1f95 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -298,7 +298,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --driver-java-options Extra Java options to pass to the driver | --driver-library-path Extra library path entries to pass to the driver - | --driver-class-path Extra class path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver. Note that + | jars added with --jars are automatically included in the + | classpath. | | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 9103c885fa96c..3e615e753b342 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -48,7 +48,8 @@ object CommandUtils extends Logging { def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") // Note, this will coalesce multiple options into a single command component - val extraOpts = command.extraJavaOptions.toSeq + val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + val libraryOpts = if (command.libraryPathEntries.size > 0) { val joined = command.libraryPathEntries.mkString(File.pathSeparator) @@ -62,10 +63,10 @@ object CommandUtils extends Logging { val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) - val userClassPath = command.classPathEntries.mkString(File.pathSeparator) - val classPathWithUser = classPath + File.pathSeparator + userClassPath + val userClassPath = command.classPathEntries ++ Seq(classPath) - Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts + Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ + libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 3836bf219ed3e..dc5158102054e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** - * Command-line parser for the master. + * Command-line parser for the worker. */ private[spark] class WorkerArguments(args: Array[String]) { var host = Utils.localHostName() diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index a1dfc4094cca7..9af9e2e8e9e59 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -106,7 +106,6 @@ private[spark] class SecurityMessage() extends Logging { * @return BufferMessage */ def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index b5111891ed35b..af26c3d59ac02 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -61,7 +61,6 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { - val p = outputsMerged.toDouble / totalOutputs val studentTCacher = new StudentTCacher(confidence) val result = new JHashMap[T, BoundedDouble](sums.size) val iter = sums.entrySet.iterator() diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 2fe65cd944b67..d822a8e55111a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} @@ -54,7 +55,7 @@ private[spark] class EventLoggingListener( private val logger = new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, - shouldOverwrite) + shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) /** * Begin logging events. @@ -124,6 +125,9 @@ private[spark] object EventLoggingListener extends Logging { val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + val LOG_FILE_PERMISSIONS: FsPermission = + FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) + // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index ace9cd51c96b7..408a797088059 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -118,11 +118,9 @@ object BlockFetcherIterator { }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { - val fetchDone = System.currentTimeMillis() val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -148,6 +146,12 @@ object BlockFetcherIterator { } protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -159,11 +163,6 @@ object BlockFetcherIterator { _numBlocksToFetch += localBlocksToFetch.size } else { numRemote += blockInfos.size - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -178,11 +177,12 @@ object BlockFetcherIterator { } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } - if (curRequestSize >= minRequestSize) { + if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") } } // Add in the final request @@ -191,7 +191,7 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -226,8 +226,8 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteRequests.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -327,7 +327,7 @@ object BlockFetcherIterator { } copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks 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 02ba5ecf52459..6d7d4f922e1fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -46,11 +46,12 @@ private[spark] class BlockManager( val master: BlockManagerMaster, val defaultSerializer: Serializer, maxMemory: Long, - val conf: SparkConf, + val _conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) extends Logging { + def conf = _conf val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 36ee4bcc41c66..0ab9fad422717 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -33,6 +33,8 @@ import org.apache.spark.util.Utils private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { + val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) + override def getSize(blockId: BlockId): Long = { diskManager.getBlockLocation(blockId).length } @@ -94,12 +96,20 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val segment = diskManager.getBlockLocation(blockId) val channel = new RandomAccessFile(segment.file, "r").getChannel() - val buffer = try { - channel.map(MapMode.READ_ONLY, segment.offset, segment.length) + + try { + // For small files, directly read rather than memory map + if (segment.length < minMemoryMapBytes) { + val buf = ByteBuffer.allocate(segment.length.toInt) + channel.read(buf, segment.offset) + buf.flip() + Some(buf) + } else { + Some(channel.map(MapMode.READ_ONLY, segment.offset, segment.length)) + } } finally { channel.close() } - Some(buffer) } override def getValues(blockId: BlockId): Option[Iterator[Any]] = { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 1ed3b70bb24fd..0965e0f0f7828 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -24,6 +24,7 @@ import java.util.Date import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec @@ -42,7 +43,8 @@ private[spark] class FileLogger( hadoopConfiguration: Configuration, outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, - overwrite: Boolean = true) + overwrite: Boolean = true, + dirPermissions: Option[FsPermission] = None) extends Logging { private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -79,16 +81,25 @@ private[spark] class FileLogger( if (!fileSystem.mkdirs(path)) { throw new IOException("Error in creating log directory: %s".format(logDir)) } + if (dirPermissions.isDefined) { + val fsStatus = fileSystem.getFileStatus(path) + if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort) { + fileSystem.setPermission(path, dirPermissions.get) + } + } } /** * Create a new writer for the file identified by the given path. + * If the permissions are not passed in, it will default to use the permissions + * (dirpermissions) used when class was instantiated. */ - private def createWriter(fileName: String): PrintWriter = { + private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme val isDefaultLocal = (defaultFs == null || defaultFs == "file") + val path = new Path(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ @@ -97,11 +108,11 @@ private[spark] class FileLogger( // Second parameter is whether to append new FileOutputStream(uri.getPath, !overwrite) } else { - val path = new Path(logPath) hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get } + perms.orElse(dirPermissions).foreach {p => fileSystem.setPermission(path, p)} val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) @@ -152,15 +163,16 @@ private[spark] class FileLogger( /** * Start a writer for a new file, closing the existing one if it exists. * @param fileName Name of the new file, defaulting to the file index if not provided. + * @param perms Permissions to put on the new file. */ - def newFile(fileName: String = "") { + def newFile(fileName: String = "", perms: Option[FsPermission] = None) { fileIndex += 1 writer.foreach(_.close()) val name = fileName match { case "" => fileIndex.toString case _ => fileName } - writer = Some(createWriter(name)) + writer = Some(createWriter(name, perms)) } /** 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 75cf7d38f6c51..2c934a4bac5d5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -28,6 +28,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag +import scala.util.Try import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -136,6 +137,11 @@ private[spark] object Utils extends Logging { def getContextOrSparkClassLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** Determines whether the provided class is loadable in the current thread. */ + def classIsLoadable(clazz: String): Boolean = { + Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess + } + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ @@ -552,8 +558,7 @@ private[spark] object Utils extends Logging { } /** - * Return the string to tell how long has passed in seconds. The passing parameter should be in - * millisecond. + * Return the string to tell how long has passed in milliseconds. */ def getUsedTimeMs(startTimeMs: Long): String = { " " + (System.currentTimeMillis - startTimeMs) + " ms" diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 7f220383f9f8b..55b5713706178 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -91,7 +91,7 @@ private[spark] object XORShiftRandom { val xorRand = new XORShiftRandom(seed) // this is just to warm up the JIT - we're not timing anything - timeIt(1e6.toInt) { + timeIt(million) { javaRand.nextInt() xorRand.nextInt() } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 76c6f5af82a5e..c3e03cea917b3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -182,13 +182,30 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } - @Test - public void toLocalIterator() { - List correct = Arrays.asList(1, 2, 3, 4); - JavaRDD rdd = sc.parallelize(correct); - List result = Lists.newArrayList(rdd.toLocalIterator()); - Assert.assertTrue(correct.equals(result)); - } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + + @Test + public void zipWithUniqueId() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); + JavaRDD indexes = zip.values(); + Assert.assertTrue(new HashSet(indexes.collect()).size() == 4); + } + + @Test + public void zipWithIndex() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); + JavaRDD indexes = zip.values(); + List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); + Assert.assertTrue(indexes.collect().equals(correctIndexes)); + } @SuppressWarnings("unchecked") @Test diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 10a65c75cc621..b3541b4a40b79 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -122,7 +122,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--num-executors 6") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) } test("handles YARN client mode") { @@ -146,6 +146,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") sysProps("spark.executor.instances") should be ("6") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles standalone cluster mode") { @@ -159,7 +160,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (1) // contains --jar entry + sysProps should have size (2) // contains --jar entry and SPARK_SUBMIT } test("handles standalone client mode") { 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 907428db80af3..00deecc1c3ca9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.storage -import java.nio.ByteBuffer +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.Arrays import akka.actor._ -import org.scalatest.BeforeAndAfter -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ @@ -785,6 +788,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + var counter = 0.toByte + def incr = {counter = (counter + 1).toByte; counter;} + val bytes = Array.fill[Byte](1000)(incr) + val byteBuffer = ByteBuffer.wrap(bytes) + + val blockId = BlockId("rdd_1_2") + + // This sequence of mocks makes these tests fairly brittle. It would + // be nice to refactor classes involved in disk storage in a way that + // allows for easier testing. + val blockManager = mock(classOf[BlockManager]) + val shuffleBlockManager = mock(classOf[ShuffleBlockManager]) + when(shuffleBlockManager.conf).thenReturn(conf) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, + System.getProperty("java.io.tmpdir")) + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) + val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val mapped = diskStoreMapped.getBytes(blockId).get + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, (1000 * 1000).toString)) + val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val notMapped = diskStoreNotMapped.getBytes(blockId).get + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + val mappedAsArray = arrayFromByteBuffer(mapped) + val notMappedAsArray = arrayFromByteBuffer(notMapped) + assert(Arrays.equals(mappedAsArray, bytes)) + assert(Arrays.equals(notMappedAsArray, bytes)) + } + test("updated block statuses") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index f1aa0f8f99e13..33552a74920a7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -21,47 +21,53 @@ # Publishes releases to Maven and packages/copies binary release artifacts. # Expects to be run in a totally empty directory. # +# Options: +# --package-only only packages an existing release candidate +# # Would be nice to add: # - Send output to stderr and have useful logging in stdout -# - Have this use sbt rather than Maven release plug in -GIT_USERNAME=pwendell -GIT_PASSWORD=XXX -GPG_PASSPHRASE=XXX -GIT_BRANCH=branch-0.9 -RELEASE_VERSION=0.9.0-incubating -RC_NAME=rc2 -USER_NAME=pwendell +GIT_USERNAME=${GIT_USERNAME:-pwendell} +GIT_PASSWORD=${GIT_PASSWORD:-XXX} +GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} +GIT_BRANCH=${GIT_BRANCH:-branch-1.0} +RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} +RC_NAME=${RC_NAME:-rc2} +USER_NAME=${USER_NAME:-pwendell} set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME -# Artifact publishing - -git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH -cd spark -export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - -mvn -Pyarn release:clean - -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - --batch-mode release:prepare +if [[ ! "$@" =~ --package-only ]]; then + echo "Creating and publishing release" + # Artifact publishing + git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH + cd spark + export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" + + mvn -Pyarn release:clean + + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + --batch-mode release:prepare + + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ + release:perform -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ - release:perform - -rm -rf spark + cd .. + rm -rf spark +fi # Source and binary tarballs +echo "Packaging release tarballs" git clone https://git-wip-us.apache.org/repos/asf/spark.git cd spark git checkout --force $GIT_TAG @@ -109,9 +115,9 @@ make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_folder scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ @@ -121,7 +127,9 @@ cd docs PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs -rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_docs_folder +rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" echo "Git tag:\t $GIT_TAG" diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 7a61943e94814..83618c8068d35 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -26,19 +26,33 @@ import json import os +import re import subprocess import sys import tempfile import urllib2 +try: + import jira.client + JIRA_IMPORTED=True +except ImportError: + JIRA_IMPORTED=False + # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") - -GIT_API_BASE = "https://api.github.com/repos/apache/spark" +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") + +GITHUB_BASE = "https://github.com/apache/spark/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" @@ -145,8 +159,7 @@ def merge_pr(pr_num, target_ref): return merge_hash -def maybe_cherry_pick(pr_num, merge_hash, default_branch): - continue_maybe("Would you like to pick %s into another branch?" % merge_hash) +def cherry_pick(pr_num, merge_hash, default_branch): pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) if pick_ref == "": pick_ref = default_branch @@ -171,14 +184,86 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) print("Pick hash: %s" % pick_hash) + return pick_ref + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + +def resolve_jira(title, merge_branches, comment): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id -branches = get_json("%s/branches" % GIT_API_BASE) + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee == None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key = lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] == False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + +branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) # Assumes branch names can be sorted lexicographically latest_branch = sorted(branch_names, reverse=True)[0] pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GIT_API_BASE, pr_num)) +pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -208,11 +293,22 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) -print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( +print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) +merged_refs = [target_ref] + merge_hash = merge_pr(pr_num, target_ref) -while True: - maybe_cherry_pick(pr_num, merge_hash, latest_branch) +pick_prompt = "Would you like to pick %s into another branch?" % merge_hash +while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + +if JIRA_IMPORTED: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) +else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA." diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 771d1933a13ea..a5e5303467401 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -39,7 +39,10 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Cloudera CDH 4.2.0 with MapReduce v1 $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property: + # Apache Hadoop 0.23.x + $ mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: # Apache Hadoop 2.0.5-alpha $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package @@ -47,9 +50,12 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with # Cloudera CDH 4.2.0 with MapReduce v2 $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package - # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer + # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + # Apache Hadoop 0.23.x + $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + ## Spark Tests in Maven ## Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. diff --git a/docs/configuration.md b/docs/configuration.md index 8d3442625b475..b078c7c1112c2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -131,6 +131,15 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + spark.storage.memoryMapThreshold + 8192 + + Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 98233bf556b79..98c456228af9f 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -63,6 +63,9 @@ All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge. Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. +# Running PySpark on YARN + +To run PySpark against a YARN cluster, simply set the MASTER environment variable to "yarn-client". # Interactive Use diff --git a/docs/security.md b/docs/security.md index 9e4218fbcfe7d..90c69915f517f 100644 --- a/docs/security.md +++ b/docs/security.md @@ -7,6 +7,8 @@ Spark currently supports authentication via a shared secret. Authentication can The Spark UI can also be secured by using javax servlet filters. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view acls to make sure they are authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' control the behavior of the acls. Note that the person who started the application always has view access to the UI. +If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secure, the permissions should be set to drwxrwxrwxt for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. + For Spark on Yarn deployments, configuring `spark.authenticate` to true will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. The Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. For other types of Spark deployments, the spark config `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. The UI can be secured using a javax servlet filter installed via `spark.ui.filters`. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. diff --git a/examples/pom.xml b/examples/pom.xml index 704d6df7c57f9..e1fc149d87f17 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -31,20 +31,6 @@ Spark Project Examples http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark @@ -124,6 +110,10 @@ commons-logging commons-logging + + org.jruby + jruby-complete + @@ -176,6 +166,11 @@ + + com.github.scopt + scopt_${scala.binary.version} + 3.2.0 + diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 2a4278d3c30e5..3f7a879538016 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -75,7 +75,6 @@ public String toString() { public static Tuple3 extractKey(String line) { Matcher m = apacheLogRegex.matcher(line); - List key = Collections.emptyList(); if (m.find()) { String ip = m.group(1); String user = m.group(3); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 1d776940f06c6..d66b9ba265fe8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -85,7 +85,7 @@ public Tuple2 call(Tuple2 e) { } }); - long oldCount = 0; + long oldCount; long nextCount = tc.count(); do { oldCount = nextCount; diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index b5b438e9753a7..d62a72f53443c 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -23,7 +23,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.sql.api.java.JavaSQLContext; import org.apache.spark.sql.api.java.JavaSchemaRDD; diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java new file mode 100644 index 0000000000000..a94fa621dc328 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java @@ -0,0 +1,152 @@ +/* + * 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.streaming.examples; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.receiver.Receiver; +import scala.Tuple2; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.net.ConnectException; +import java.net.Socket; +import java.util.regex.Pattern; + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: JavaCustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + */ + +public class JavaCustomReceiver extends Receiver { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 3) { + System.err.println("Usage: JavaNetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Create the context with a 1 second batch size + JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", + new Duration(1000), System.getenv("SPARK_HOME"), + JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + JavaDStream lines = ssc.receiverStream( + new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + wordCounts.print(); + ssc.start(); + ssc.awaitTermination(); + } + + // ============= Receiver code that receives data over a socket ============== + + String host = null; + int port = -1; + + public JavaCustomReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK_2()); + host = host_; + port = port_; + } + + public void onStart() { + // Start the thread that receives data over a connection + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + public void onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private void receive() { + Socket socket = null; + String userInput = null; + + try { + // connect to the server + socket = new Socket(host, port); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + + // Until stopped or connection broken continue reading + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + reader.close(); + socket.close(); + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again"); + } catch(ConnectException ce) { + // restart if could not connect to server + restart("Could not connect", ce); + } catch(Throwable t) { + restart("Error receiving data", t); + } + } +} + + diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 7f68d451e9b31..0cc9d0ae1a08e 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -31,7 +31,7 @@ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCount + * Usage: JavaNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. * @@ -43,9 +43,6 @@ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - private JavaNetworkWordCount() { - } - public static void main(String[] args) { if (args.length < 3) { System.err.println("Usage: JavaNetworkWordCount \n" + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala new file mode 100644 index 0000000000000..ec9de022c1d47 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -0,0 +1,145 @@ +/* + * 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.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} + +/** + * An example app for binary classification. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.BinaryClassification + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object BinaryClassification { + + object Algorithm extends Enumeration { + type Algorithm = Value + val SVM, LR = Value + } + + object RegType extends Enumeration { + type RegType = Value + val L1, L2 = Value + } + + import Algorithm._ + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + algorithm: Algorithm = LR, + regType: RegType = L2, + regParam: Double = 0.1) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("BinaryClassification") { + head("BinaryClassification: an example app for binary classification.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("algorithm") + .text(s"algorithm (${Algorithm.values.mkString(",")}), " + + s"default: ${defaultParams.algorithm}") + .action((x, c) => c.copy(algorithm = Algorithm.withName(x))) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"BinaryClassification with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val model = params.algorithm match { + case LR => + val algorithm = new LogisticRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + case SVM => + val algorithm = new SVMWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + algorithm.run(training).clearThreshold() + } + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val metrics = new BinaryClassificationMetrics(predictionAndLabel) + + println(s"Test areaUnderPR = ${metrics.areaUnderPR()}.") + println(s"Test areaUnderROC = ${metrics.areaUnderROC()}.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala new file mode 100644 index 0000000000000..0bd847d7bab30 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -0,0 +1,161 @@ +/* + * 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.mllib + +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree, impurity} +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +/** + * An example runner for decision tree. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeRunner { + + object ImpurityType extends Enumeration { + type ImpurityType = Value + val Gini, Entropy, Variance = Value + } + + import ImpurityType._ + + case class Params( + input: String = null, + algo: Algo = Classification, + maxDepth: Int = 5, + impurity: ImpurityType = Gini, + maxBins: Int = 20) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeRunner") { + head("DecisionTreeRunner: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (${Algo.values.mkString(",")}), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = Algo.withName(x))) + opt[String]("impurity") + .text(s"impurity type (${ImpurityType.values.mkString(",")}), " + + s"default: ${defaultParams.impurity}") + .action((x, c) => c.copy(impurity = ImpurityType.withName(x))) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + arg[String]("") + .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.algo == Classification && + (params.impurity == Gini || params.impurity == Entropy)) { + success + } else if (params.algo == Regression && params.impurity == Variance) { + success + } else { + failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName("DecisionTreeRunner") + val sc = new SparkContext(conf) + + // Load training data and cache it. + val examples = MLUtils.loadLabeledData(sc, params.input).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + examples.unpersist(blocking = false) + + val impurityCalculator = params.impurity match { + case Gini => impurity.Gini + case Entropy => impurity.Entropy + case Variance => impurity.Variance + } + + val strategy = new Strategy(params.algo, impurityCalculator, params.maxDepth, params.maxBins) + val model = DecisionTree.train(training, strategy) + + if (params.algo == Classification) { + val accuracy = accuracyScore(model, test) + println(s"Test accuracy = $accuracy.") + } + + if (params.algo == Regression) { + val mse = meanSquaredError(model, test) + println(s"Test mean squared error = $mse.") + } + + sc.stop() + } + + /** + * Calculates the classifier accuracy. + */ + private def accuracyScore( + model: DecisionTreeModel, + data: RDD[LabeledPoint], + threshold: Double = 0.5): Double = { + def predictedValue(features: Vector): Double = { + if (model.predict(features) < threshold) 0.0 else 1.0 + } + val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + val count = data.count() + correctCount.toDouble / count + } + + /** + * Calculates the mean squared error for regression. + */ + private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala new file mode 100644 index 0000000000000..f96bc1bf00b92 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -0,0 +1,109 @@ +/* + * 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.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.KMeans +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example k-means app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.DenseKMeans [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseKMeans { + + object InitializationMode extends Enumeration { + type InitializationMode = Value + val Random, Parallel = Value + } + + import InitializationMode._ + + case class Params( + input: String = null, + k: Int = -1, + numIterations: Int = 10, + initializationMode: InitializationMode = Parallel) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DenseKMeans") { + head("DenseKMeans: an example k-means app for dense data.") + opt[Int]('k', "k") + .required() + .text(s"number of clusters, required") + .action((x, c) => c.copy(k = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default; ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[String]("initMode") + .text(s"initialization mode (${InitializationMode.values.mkString(",")}), " + + s"default: ${defaultParams.initializationMode}") + .action((x, c) => c.copy(initializationMode = InitializationMode.withName(x))) + arg[String]("") + .text("input paths to examples") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DenseKMeans with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = sc.textFile(params.input).map { line => + Vectors.dense(line.split(' ').map(_.toDouble)) + }.cache() + + val numExamples = examples.count() + + println(s"numExamples = $numExamples.") + + val initMode = params.initializationMode match { + case Random => KMeans.RANDOM + case Parallel => KMeans.K_MEANS_PARALLEL + } + + val model = new KMeans() + .setInitializationMode(initMode) + .setK(params.k) + .setMaxIterations(params.numIterations) + .run(examples) + + val cost = model.computeCost(examples) + + println(s"Total cost = $cost.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala new file mode 100644 index 0000000000000..1723ca6931021 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -0,0 +1,125 @@ +/* + * 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.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils} +import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} + +/** + * An example app for linear regression. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.LinearRegression + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LinearRegression extends App { + + object RegType extends Enumeration { + type RegType = Value + val NONE, L1, L2 = Value + } + + import RegType._ + + case class Params( + input: String = null, + numIterations: Int = 100, + stepSize: Double = 1.0, + regType: RegType = L2, + regParam: Double = 0.1) + + val defaultParams = Params() + + val parser = new OptionParser[Params]("LinearRegression") { + head("LinearRegression: an example app for linear regression.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"LinearRegression with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + examples.unpersist(blocking = false) + + val updater = params.regType match { + case NONE => new SimpleUpdater() + case L1 => new L1Updater() + case L2 => new SquaredL2Updater() + } + + val algorithm = new LinearRegressionWithSGD() + algorithm.optimizer + .setNumIterations(params.numIterations) + .setStepSize(params.stepSize) + .setUpdater(updater) + .setRegParam(params.regParam) + + val model = algorithm.run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + + val loss = predictionAndLabel.map { case (p, l) => + val err = p - l + err * err + }.reduce(_ + _) + val rmse = math.sqrt(loss / numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala new file mode 100644 index 0000000000000..703f02255b94b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -0,0 +1,131 @@ +/* + * 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.mllib + +import com.esotericsoftware.kryo.Kryo +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.recommendation.{ALS, MatrixFactorizationModel, Rating} +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} + +/** + * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). + */ +object MovieLensALS { + + class ALSRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[Rating]) + } + } + + case class Params( + input: String = null, + kryo: Boolean = false, + numIterations: Int = 20, + lambda: Double = 1.0, + rank: Int = 10) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("MovieLensALS") { + head("MovieLensALS: an example app for ALS on MovieLens data.") + opt[Int]("rank") + .text(s"rank, default: ${defaultParams.rank}}") + .action((x, c) => c.copy(rank = x)) + opt[Int]("numIterations") + .text(s"number of iterations, default: ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + opt[Unit]("kryo") + .text(s"use Kryo serialization") + .action((_, c) => c.copy(kryo = true)) + arg[String]("") + .required() + .text("input paths to a MovieLens dataset of ratings") + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + System.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"MovieLensALS with $params") + if (params.kryo) { + conf.set("spark.serializer", classOf[KryoSerializer].getName) + .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + .set("spark.kryoserializer.buffer.mb", "8") + } + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val ratings = sc.textFile(params.input).map { line => + val fields = line.split("::") + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + }.cache() + + val numRatings = ratings.count() + val numUsers = ratings.map(_.user).distinct().count() + val numMovies = ratings.map(_.product).distinct().count() + + println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") + + val splits = ratings.randomSplit(Array(0.8, 0.2)) + val training = splits(0).cache() + val test = splits(1).cache() + + val numTraining = training.count() + val numTest = test.count() + println(s"Training: $numTraining, test: $numTest.") + + ratings.unpersist(blocking = false) + + val model = new ALS() + .setRank(params.rank) + .setIterations(params.numIterations) + .setLambda(params.lambda) + .run(training) + + val rmse = computeRmse(model, test, numTest) + + println(s"Test RMSE = $rmse.") + + sc.stop() + } + + /** Compute RMSE (Root Mean Squared Error). */ + def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], n: Long) = { + val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) + val predictionsAndRatings = predictions.map(x => ((x.user, x.product), x.rating)) + .join(data.map(x => ((x.user, x.product), x.rating))) + .values + math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).reduce(_ + _) / n) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala new file mode 100644 index 0000000000000..25b6768b8d72b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -0,0 +1,102 @@ +/* + * 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.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.classification.NaiveBayes +import org.apache.spark.mllib.util.{MLUtils, MulticlassLabelParser} + +/** + * An example naive Bayes app. Run with + * {{{ + * ./bin/spark-example org.apache.spark.examples.mllib.SparseNaiveBayes [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object SparseNaiveBayes { + + case class Params( + input: String = null, + minPartitions: Int = 0, + numFeatures: Int = -1, + lambda: Double = 1.0) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("SparseNaiveBayes") { + head("SparseNaiveBayes: an example naive Bayes app for LIBSVM data.") + opt[Int]("numPartitions") + .text("min number of partitions") + .action((x, c) => c.copy(minPartitions = x)) + opt[Int]("numFeatures") + .text("number of features") + .action((x, c) => c.copy(numFeatures = x)) + opt[Double]("lambda") + .text(s"lambda (smoothing constant), default: ${defaultParams.lambda}") + .action((x, c) => c.copy(lambda = x)) + arg[String]("") + .text("input paths to labeled examples in LIBSVM format") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"SparseNaiveBayes with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val minPartitions = + if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions + + val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser, + params.numFeatures, minPartitions) + // Cache examples because it will be used in both training and evaluation. + examples.cache() + + val splits = examples.randomSplit(Array(0.8, 0.2)) + val training = splits(0) + val test = splits(1) + + val numTraining = training.count() + val numTest = test.count() + + println(s"numTraining = $numTraining, numTest = $numTest.") + + val model = new NaiveBayes().setLambda(params.lambda).run(training) + + val prediction = model.predict(test.map(_.features)) + val predictionAndLabel = prediction.zip(test.map(_.label)) + val accuracy = predictionAndLabel.filter(x => x._1 == x._2).count().toDouble / numTest + + println(s"Test accuracy = $accuracy.") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 39e71cdab4328..3cd9cb743e309 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnyPCA { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnyPCA ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnyPCA ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnyPCA") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass).toSeq) + val conf = new SparkConf().setAppName("TallSkinnyPCA") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 2b7de2acc65d6..4d6690318615a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -35,20 +35,16 @@ import org.apache.spark.mllib.linalg.Vectors */ object TallSkinnySVD { def main(args: Array[String]) { - if (args.length != 2) { - System.err.println("Usage: TallSkinnySVD ") + if (args.length != 1) { + System.err.println("Usage: TallSkinnySVD ") System.exit(1) } - val conf = new SparkConf() - .setMaster(args(0)) - .setAppName("TallSkinnySVD") - .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass).toSeq) + val conf = new SparkConf().setAppName("TallSkinnySVD") val sc = new SparkContext(conf) // Load and parse the data file. - val rows = sc.textFile(args(1)).map { line => + val rows = sc.textFile(args(0)).map { line => val values = line.split(' ').map(_.toDouble) Vectors.dense(values) } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala new file mode 100644 index 0000000000000..eebffd824983f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala @@ -0,0 +1,108 @@ +/* + * 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.streaming.examples + +import java.io.{InputStreamReader, BufferedReader, InputStream} +import java.net.Socket + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.receiver.Receiver + +/** + * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * text and \n delimited lines are considered as records. They are then counted and printed. + * + * Usage: CustomReceiver + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and of the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + */ +object CustomReceiver { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + + // Create a input stream with the custom receiver on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + val lines = ssc.receiverStream(new CustomReceiver(args(1), args(2).toInt)) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.print() + ssc.start() + ssc.awaitTermination() + } +} + + +class CustomReceiver(host: String, port: Int) + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + override def run() { receive() } + }.start() + } + + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private def receive() { + var socket: Socket = null + var userInput: String = null + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + logInfo("Connected to " + host + ":" + port) + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + logInfo("Stopped receiving") + restart("Trying to connect again") + } catch { + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } + } +} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b84ca0c5e1dd0..03d3b2394f510 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -31,20 +31,6 @@ Spark Project External Flume http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 343e1fabd823f..979eb0ca624bd 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -31,20 +31,6 @@ Spark Project External Kafka http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 3710a63541d78..9aa1c1a9f5b80 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -31,20 +31,6 @@ Spark Project External MQTT http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 398b9f4fbaa7d..a443459594710 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -31,20 +31,6 @@ Spark Project External Twitter http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 77e957f404645..a40e55876e640 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -31,20 +31,6 @@ Spark Project External ZeroMQ http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/graphx/pom.xml b/graphx/pom.xml index b4c67ddcd8ca9..dc108d2fe7fbd 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -31,20 +31,6 @@ Spark Project GraphX http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/make-distribution.sh b/make-distribution.sh index 55fe6c09d0d66..c05dcd89d90a7 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -43,7 +43,7 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -VERSION=$(mvn help:evaluate -Dexpression=project.version |grep -v "INFO") +VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) if [ $? == -1 ] ;then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org." @@ -123,10 +123,19 @@ else fi if [ "$SPARK_YARN" == "true" ]; then - mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23 + else + mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi else - mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then + mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + else + mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE + fi fi # Make directories @@ -178,6 +187,7 @@ fi if [ "$MAKE_TGZ" == "true" ]; then TARDIR_NAME=spark-$VERSION-bin-$NAME TARDIR="$FWDIR/$TARDIR_NAME" + rm -rf "$TARDIR" cp -r "$DISTDIR" "$TARDIR" tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME" rm -rf "$TARDIR" diff --git a/mllib/pom.xml b/mllib/pom.xml index e7ce00efc4af6..cdd33dbb7970d 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -31,20 +31,6 @@ Spark Project ML Library http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 4f9eaacf67fe4..780e8bae42b84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object LogisticRegressionWithSGD { numIterations: Int): LogisticRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression " + - "") - System.exit(1) - } - val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 18658850a2f64..f6f62ce2de04e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -158,23 +157,4 @@ object NaiveBayes { def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { new NaiveBayes(lambda).run(input) } - - def main(args: Array[String]) { - if (args.length != 2 && args.length != 3) { - println("Usage: NaiveBayes []") - System.exit(1) - } - val sc = new SparkContext(args(0), "NaiveBayes") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = if (args.length == 2) { - NaiveBayes.train(data) - } else { - NaiveBayes.train(data, args(2).toDouble) - } - - println("Pi\n: " + model.pi) - println("Theta:\n" + model.theta) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 956654b1fe90a..81b126717e9a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.classification -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.DataValidators import org.apache.spark.rdd.RDD /** @@ -183,19 +182,4 @@ object SVMWithSGD { def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: SVM ") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index dee9ef07e41ed..a64c5d44be406 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,8 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} -import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils @@ -396,28 +395,6 @@ object KMeans { v2: BreezeVectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - - @Experimental - def main(args: Array[String]) { - if (args.length < 4) { - println("Usage: KMeans []") - System.exit(1) - } - val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) - val runs = if (args.length >= 5) args(4).toInt else 1 - val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile) - .map(line => Vectors.dense(line.split(' ').map(_.toDouble))) - .cache() - val model = KMeans.train(data, k, iters, runs) - val cost = model.computeCost(data) - println("Cluster centers:") - for (c <- model.clusterCenters) { - println(" " + c) - } - println("Cost: " + cost) - System.exit(0) - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 60fb73f2b5be5..0cf9a7f909081 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -23,15 +23,13 @@ import scala.util.Random import scala.util.Sorting import scala.util.hashing.byteswap32 -import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast -import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} +import org.apache.spark.{Logging, HashPartitioner, Partitioner} import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD -import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils @@ -474,13 +472,15 @@ class ALS private ( // Compute the XtX and Xy values for each user by adding products it rated in each product // block for (productBlock <- 0 until numBlocks) { - for (p <- 0 until blockFactors(productBlock).length) { + var p = 0 + while (p < blockFactors(productBlock).length) { val x = wrapDoubleArray(blockFactors(productBlock)(p)) tempXtX.fill(0.0) dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) - for (i <- 0 until us.length) { - if (implicitPrefs) { + if (implicitPrefs) { + var i = 0 + while (i < us.length) { // Extension to the original paper to handle rs(i) < 0. confidence is a function // of |rs(i)| instead so that it is never negative: val confidence = 1 + alpha * abs(rs(i)) @@ -491,11 +491,17 @@ class ALS private ( if (rs(i) > 0) { SimpleBlas.axpy(confidence, x, userXy(us(i))) } - } else { + i += 1 + } + } else { + var i = 0 + while (i < us.length) { userXtX(us(i)).addi(tempXtX) SimpleBlas.axpy(rs(i), x, userXy(us(i))) + i += 1 } } + p += 1 } } @@ -504,7 +510,11 @@ class ALS private ( // Compute the full XtX matrix from the lower-triangular part we got above fillFullMatrix(userXtX(index), fullXtX) // Add regularization - (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) + var i = 0 + while (i < rank) { + fullXtX.data(i * rank + i) += lambda + i += 1 + } // Solve the resulting matrix, which is symmetric and positive-definite if (implicitPrefs) { Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data @@ -707,45 +717,4 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } - - private class ALSRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[Rating]) - } - } - - def main(args: Array[String]) { - if (args.length < 5 || args.length > 9) { - println("Usage: ALS " + - "[] [] [] []") - System.exit(1) - } - val (master, ratingsFile, rank, iters, outputDir) = - (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) - val lambda = if (args.length >= 6) args(5).toDouble else 0.01 - val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false - val alpha = if (args.length >= 8) args(7).toDouble else 1 - val blocks = if (args.length == 9) args(8).toInt else -1 - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) - .set("spark.kryo.referenceTracking", "false") - .set("spark.kryoserializer.buffer.mb", "8") - .set("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS", conf) - - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = new ALS(rank = rank, iterations = iters, lambda = lambda, - numBlocks = blocks, implicitPrefs = implicitPrefs, alpha = alpha).run(ratings) - - model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/userFeatures") - model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/productFeatures") - println("Final user/product features written to " + outputDir) - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 5f0812fd2e0eb..0e6fb1b1caa41 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** @@ -173,19 +171,4 @@ object LassoWithSGD { numIterations: Int): LassoModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: Lasso ") - System.exit(1) - } - val sc = new SparkContext(args(0), "Lasso") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 228fa8db3e721..1532ff90d846d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,11 +17,9 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils /** * Regression model trained using LinearRegression. @@ -156,18 +154,4 @@ object LinearRegressionWithSGD { numIterations: Int): LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LinearRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "LinearRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index e702027c7c170..5f7e25a9b8be1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.linalg.Vector /** @@ -170,21 +168,4 @@ object RidgeRegressionWithSGD { numIterations: Int): RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, - args(3).toDouble) - - println("Weights: " + model.weights) - println("Intercept: " + model.intercept) - - sc.stop() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index f68076f426259..59ed01debf150 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -18,18 +18,16 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Experimental -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: Experimental :: @@ -1028,129 +1026,4 @@ object DecisionTree extends Serializable with Logging { throw new UnsupportedOperationException("approximate histogram not supported yet.") } } - - private val usage = """ - Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] - """ - - def main(args: Array[String]) { - - if (args.length < 2) { - System.err.println(usage) - System.exit(1) - } - - val sc = new SparkContext(args(0), "DecisionTree") - - val argList = args.toList.drop(1) - type OptionMap = Map[Symbol, Any] - - def nextOption(map : OptionMap, list: List[String]): OptionMap = { - list match { - case Nil => map - case "--algo" :: string :: tail => nextOption(map ++ Map('algo -> string), tail) - case "--impurity" :: string :: tail => nextOption(map ++ Map('impurity -> string), tail) - case "--maxDepth" :: string :: tail => nextOption(map ++ Map('maxDepth -> string), tail) - case "--maxBins" :: string :: tail => nextOption(map ++ Map('maxBins -> string), tail) - case "--trainDataDir" :: string :: tail => nextOption(map ++ Map('trainDataDir -> string) - , tail) - case "--testDataDir" :: string :: tail => nextOption(map ++ Map('testDataDir -> string), - tail) - case string :: Nil => nextOption(map ++ Map('infile -> string), list.tail) - case option :: tail => logError("Unknown option " + option) - sys.exit(1) - } - } - val options = nextOption(Map(), argList) - logDebug(options.toString()) - - // Load training data. - val trainData = loadLabeledData(sc, options.get('trainDataDir).get.toString) - - // Identify the type of algorithm. - val algoStr = options.get('algo).get.toString - val algo = algoStr match { - case "Classification" => Classification - case "Regression" => Regression - } - - // Identify the type of impurity. - val impurityStr = options.getOrElse('impurity, - if (algo == Classification) "Gini" else "Variance").toString - val impurity = impurityStr match { - case "Gini" => Gini - case "Entropy" => Entropy - case "Variance" => Variance - } - - val maxDepth = options.getOrElse('maxDepth, "1").toString.toInt - val maxBins = options.getOrElse('maxBins, "100").toString.toInt - - val strategy = new Strategy(algo, impurity, maxDepth, maxBins) - val model = DecisionTree.train(trainData, strategy) - - // Load test data. - val testData = loadLabeledData(sc, options.get('testDataDir).get.toString) - - // Measure algorithm accuracy - if (algo == Classification) { - val accuracy = accuracyScore(model, testData) - logDebug("accuracy = " + accuracy) - } - - if (algo == Regression) { - val mse = meanSquaredError(model, testData) - logDebug("mean square error = " + mse) - } - - sc.stop() - } - - /** - * Load labeled data from a file. The data format used here is - * , ..., - * where , are feature values in Double and is the corresponding label as Double. - * - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is - * the label, and the second element represents the feature values (an array of Double). - */ - private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { - sc.textFile(dir).map { line => - val parts = line.trim().split(",") - val label = parts(0).toDouble - val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble)) - LabeledPoint(label, features) - } - } - - // TODO: Port this method to a generic metrics package. - /** - * Calculates the classifier accuracy. - */ - private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], - threshold: Double = 0.5): Double = { - def predictedValue(features: Vector) = { - if (model.predict(features) < threshold) 0.0 else 1.0 - } - val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() - val count = data.count() - logDebug("correct prediction count = " + correctCount) - logDebug("data count = " + count) - correctCount.toDouble / count - } - - // TODO: Port this method to a generic metrics package - /** - * Calculates the mean squared error for regression. - */ - private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { - data.map { y => - val err = tree.predict(y.features) - y.label - err * err - }.mean() - } } diff --git a/pom.xml b/pom.xml index e9117998252fc..646753fe30301 100644 --- a/pom.xml +++ b/pom.xml @@ -127,6 +127,7 @@ 8.1.14.v20131031 0.3.6 3.0.0 + 1.7.4 64m 512m @@ -293,7 +294,7 @@ org.ow2.asm asm-commons - + @@ -308,7 +309,7 @@ org.ow2.asm asm-commons - + @@ -425,6 +426,16 @@ scala-library ${scala.version} + + org.scala-lang + scala-actors + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + org.scalatest scalatest_${scala.binary.version} @@ -492,7 +503,45 @@ org.apache.avro avro - 1.7.4 + ${avro.version} + + + org.jboss.netty + netty + + + io.netty + netty + + + + + org.apache.avro + avro-ipc + ${avro.version} + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + + + + + org.apache.avro + avro-mapred + ${avro.version} org.jboss.netty @@ -687,7 +736,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + @@ -742,7 +791,7 @@ build-helper-maven-plugin - add-scala-sources + add-scala-sources generate-sources add-source @@ -779,6 +828,17 @@ + + + hadoop-0.23 + + + org.apache.avro + avro + + + + yarn-alpha @@ -790,6 +850,12 @@ yarn + + + org.apache.avro + avro + + @@ -838,7 +904,7 @@ - + hadoop-provided @@ -878,6 +944,7 @@ org.apache.zookeeper zookeeper + 3.4.5 provided diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 48f234ce87a44..51f7335111166 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -106,7 +106,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" - val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() + val maybeAvro = if (hadoopVersion.startsWith("0.23.")) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { case None => DEFAULT_HIVE @@ -224,7 +224,7 @@ object SparkBuild extends Build { org.apache apache - 13 + 14 http://spark.apache.org/ @@ -250,7 +250,7 @@ object SparkBuild extends Build { JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK ), @@ -313,6 +313,7 @@ object SparkBuild extends Build { val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") + val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", @@ -417,7 +418,7 @@ object SparkBuild extends Build { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -425,7 +426,8 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J) + excludeAll(excludeSLF4J), + "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings @@ -506,7 +508,7 @@ object SparkBuild extends Build { |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ - |import org.apache.spark.sql.hive.TestHive._ + |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) diff --git a/python/.gitignore b/python/.gitignore index 5c56e638f923a..80b361ffbd51c 100644 --- a/python/.gitignore +++ b/python/.gitignore @@ -1,2 +1,5 @@ *.pyc docs/ +pyspark.egg-info +build/ +dist/ diff --git a/python/lib/PY4J_VERSION.txt b/python/lib/PY4J_VERSION.txt deleted file mode 100644 index 04a0cd52a8d9c..0000000000000 --- a/python/lib/PY4J_VERSION.txt +++ /dev/null @@ -1 +0,0 @@ -b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 73fe7378ffa63..07df8697bd1a8 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -49,13 +49,6 @@ Main entry point for accessing data stored in Apache Hive.. """ - - -import sys -import os -sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip")) - - from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql import SQLContext diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6bb6c877c942d..032d960e40998 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -24,10 +24,11 @@ from py4j.java_gateway import java_import, JavaGateway, GatewayClient -SPARK_HOME = os.environ["SPARK_HOME"] +def launch_gateway(): + SPARK_HOME = os.environ["SPARK_HOME"] + set_env_vars_for_yarn() -def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" @@ -70,3 +71,27 @@ def run(self): java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext") java_import(gateway.jvm, "scala.Tuple2") return gateway + +def set_env_vars_for_yarn(): + # Add the spark jar, which includes the pyspark files, to the python path + env_map = parse_env(os.environ.get("SPARK_YARN_USER_ENV", "")) + if "PYTHONPATH" in env_map: + env_map["PYTHONPATH"] += ":spark.jar" + else: + env_map["PYTHONPATH"] = "spark.jar" + + os.environ["SPARK_YARN_USER_ENV"] = ",".join(k + '=' + v for (k, v) in env_map.items()) + +def parse_env(env_str): + # Turns a comma-separated of env settings into a dict that maps env vars to + # their values. + env = {} + for var_str in env_str.split(","): + parts = var_str.split("=") + if len(parts) == 2: + env[parts[0]] = parts[1] + elif len(var_str) > 0: + print "Invalid entry in SPARK_YARN_USER_ENV: " + var_str + sys.exit(1) + + return env diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a59778c72130e..3a1c56af5b221 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -537,8 +537,8 @@ def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. - >>> sc.parallelize([1, 2, 3]).pipe('cat').collect() - ['1', '2', '3'] + >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() + ['1', '2', '', '3'] """ def func(iterator): pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) @@ -547,7 +547,7 @@ def pipe_objs(out): out.write(str(obj).rstrip('\n') + '\n') out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in pipe.stdout) + return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) return self.mapPartitions(func) def foreach(self, f): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 114fa138d0de2..1a62031db5c41 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -143,13 +143,13 @@ def table(self, tableName): """ return SchemaRDD(self._ssql_ctx.table(tableName), self) - def cacheTable(tableName): + def cacheTable(self, tableName): """ Caches the specified table in-memory. """ self._ssql_ctx.cacheTable(tableName) - def uncacheTable(tableName): + def uncacheTable(self, tableName): """ Removes the specified table from the in-memory cache. """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 527104587fd31..8cf9d9cf1bd66 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -30,10 +30,12 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.java_gateway import SPARK_HOME from pyspark.serializers import read_int +SPARK_HOME = os.environ["SPARK_HOME"] + + class PySparkTestCase(unittest.TestCase): def setUp(self): diff --git a/repl/pom.xml b/repl/pom.xml index 78d2fe13c27eb..b761a176ce256 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -31,20 +31,6 @@ Spark Project REPL http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - /usr/share/spark root diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index cd2c7b7b0d496..147b506dd5ca3 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -34,3 +34,6 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} export SPARK_CONF_DIR="$SPARK_HOME/conf" +# Add the PySpark classes to the PYTHONPATH: +export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 9d5c6a857bb00..8d2e4baf69e30 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,67 +16,56 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-catalyst_2.10 - jar - Spark Project Catalyst - http://spark.apache.org/ + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + - - - yarn-alpha - - - org.apache.avro - avro - - - - + org.apache.spark + spark-catalyst_2.10 + jar + Spark Project Catalyst + http://spark.apache.org/ - - - org.scala-lang - scala-reflect - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.scala-lang + scala-reflect + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + 1.0.1 + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 85580ed6b822f..fb3b190b4ec5a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,72 +16,62 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-sql_2.10 - jar - Spark Project SQL - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql_2.10 + jar + Spark Project SQL + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - - - com.twitter - parquet-column - ${parquet.version} - - - com.twitter - parquet-hadoop - ${parquet.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + + + com.twitter + parquet-column + ${parquet.version} + + + com.twitter + parquet-hadoop + ${parquet.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a662da76ce25a..889d249146b8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,105 +16,95 @@ ~ limitations under the License. --> - - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../../pom.xml - - + + 4.0.0 + org.apache.spark - spark-hive_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - - yarn-alpha - - - org.apache.avro - avro - - - - + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive_2.10 + jar + Spark Project Hive + http://spark.apache.org/ - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - - - org.apache.hive - hive-metastore - ${hive.version} - - - org.apache.hive - hive-exec - ${hive.version} - - - org.codehaus.jackson - jackson-mapper-asl - - - org.apache.hive - hive-serde - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + org.apache.hive + hive-metastore + ${hive.version} + + + org.apache.hive + hive-exec + ${hive.version} + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.hive + hive-serde + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + - - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - copy-dependencies - package - - copy-dependencies - - - - ${basedir}/../../lib_managed/jars - false - false - true - org.datanucleus - - - - - - + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + + + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index a09270eb7b134..c7de4ab6d3955 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -70,24 +70,26 @@ private[hive] object HiveFunctionRegistry } def javaClassToDataType(clz: Class[_]): DataType = clz match { + // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType + case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType + + // java class case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[Array[Byte]] => BinaryType case c: Class[_] if c == classOf[java.lang.Short] => ShortType case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType case c: Class[_] if c == classOf[java.lang.Long] => LongType @@ -95,6 +97,16 @@ private[hive] object HiveFunctionRegistry case c: Class[_] if c == classOf[java.lang.Byte] => ByteType case c: Class[_] if c == classOf[java.lang.Float] => FloatType case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + + // primitive type + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) } } @@ -111,11 +123,19 @@ private[hive] trait HiveFunctionFactory { case i: hadoopIo.IntWritable => i.get case t: hadoopIo.Text => t.toString case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get() + case d: hadoopIo.DoubleWritable => d.get case d: hiveIo.DoubleWritable => d.get case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get() + case b: hadoopIo.BooleanWritable => b.get case b: hiveIo.ByteWritable => b.get + case b: hadoopIo.FloatWritable => b.get + case b: hadoopIo.BytesWritable => { + val bytes = new Array[Byte](b.getLength) + System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) + bytes + } + case t: hiveIo.TimestampWritable => t.getTimestamp + case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) case list: java.util.List[_] => list.map(unwrap) case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap case array: Array[_] => array.map(unwrap).toSeq @@ -127,6 +147,9 @@ private[hive] trait HiveFunctionFactory { case p: java.lang.Byte => p case p: java.lang.Boolean => p case str: String => str + case p: BigDecimal => p + case p: Array[Byte] => p + case p: java.sql.Timestamp => p } } @@ -252,13 +275,17 @@ private[hive] trait HiveInspectors { /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) + case s: String => new hadoopIo.Text(s) // TODO why should be Text? case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float case d: Double => d: java.lang.Double case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte + case b: BigDecimal => b.bigDecimal + case b: Array[Byte] => b + case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) @@ -280,6 +307,8 @@ private[hive] trait HiveInspectors { case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector } def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { @@ -307,6 +336,14 @@ private[hive] trait HiveInspectors { case _: JavaShortObjectInspector => ShortType case _: WritableByteObjectInspector => ByteType case _: JavaByteObjectInspector => ByteType + case _: WritableFloatObjectInspector => FloatType + case _: JavaFloatObjectInspector => FloatType + case _: WritableBinaryObjectInspector => BinaryType + case _: JavaBinaryObjectInspector => BinaryType + case _: WritableHiveDecimalObjectInspector => DecimalType + case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableTimestampObjectInspector => TimestampType + case _: JavaTimestampObjectInspector => TimestampType } implicit class typeInfoConversions(dt: DataType) { @@ -324,6 +361,7 @@ private[hive] trait HiveInspectors { case ShortType => shortTypeInfo case StringType => stringTypeInfo case DecimalType => decimalTypeInfo + case TimestampType => timestampTypeInfo case NullType => voidTypeInfo } } diff --git a/streaming/pom.xml b/streaming/pom.xml index 93b1c5a37aff9..6435224a14674 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -31,20 +31,6 @@ Spark Project Streaming http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index fbb2e9f85dd12..75a3e9334e6d5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -390,7 +390,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of Receiver */ - def receiverStream[T](receiver: Receiver[T]): ReceiverInputDStream[T] = { + def receiverStream[T](receiver: Receiver[T]): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.receiverStream(receiver) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 524c1b8d8ce46..b310c22b3ab78 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -30,28 +30,55 @@ import org.apache.spark.annotation.DeveloperApi * Abstract class of a receiver that can be run on worker nodes to receive external data. A * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() * should define the setup steps necessary to start receiving data, - * and onStop() should define the cleanup steps necessary to stop receiving data. A custom - * receiver would look something like this. + * and onStop() should define the cleanup steps necessary to stop receiving data. * - * @example {{{ + * A custom receiver in Scala would look like this. + * + * {{{ * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) { - * def onStart() { - * // Setup stuff (start threads, open sockets, etc.) to start receiving data. - * // Must start new thread to receive data, as onStart() must be non-blocking. + * def onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. * - * // Call store(...) in those threads to store received data into Spark's memory. + * // Call store(...) in those threads to store received data into Spark's memory. * - * // Call stop(...), restart() or reportError(...) on any thread based on how - * // different errors should be handled. + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. * - * // See corresponding method documentation for more details - * } + * // See corresponding method documentation for more details + * } * - * def onStop() { - * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. - * } + * def onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } * } * }}} + * + * A custom receiver in Java would look like this. + * + * {{{ + * class MyReceiver extends Receiver { + * public MyReceiver(StorageLevel storageLevel) { + * super(storageLevel); + * } + * + * public void onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart(...) or reportError(...) on any thread based on how + * // different errors needs to be handled. + * + * // See corresponding method documentation for more details + * } + * + * public void onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} */ @DeveloperApi abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { @@ -156,30 +183,34 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after a delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. * The `message` will be reported to the driver. - * The delay is defined by the Spark configuration - * `spark.streaming.receiverRestartDelay`. */ def restart(message: String) { executor.restartReceiver(message) } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after a delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. The delay between the stopping and the starting + * is defined by the Spark configuration `spark.streaming.receiverRestartDelay`. * The `message` and `exception` will be reported to the driver. - * The delay is defined by the Spark configuration - * `spark.streaming.receiverRestartDelay`. */ def restart(message: String, error: Throwable) { executor.restartReceiver(message, Some(error)) } /** - * Restart the receiver. This will call `onStop()` immediately and return. - * Asynchronously, after the given delay, `onStart()` will be called. + * Restart the receiver. This method schedules the restart and returns + * immediately. The stopping and subsequent starting of the receiver + * (by calling `onStop()` and `onStart()`) is performed asynchronously + * in a background thread. */ def restart(message: String, error: Throwable, millisecond: Int) { executor.restartReceiver(message, Some(error), millisecond) @@ -195,16 +226,23 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable executor.stop(message, Some(error)) } + /** Check if the receiver has started or not. */ def isStarted(): Boolean = { executor.isReceiverStarted() } - /** Check if receiver has been marked for stopping. */ + /** + * Check if receiver has been marked for stopping. Use this to identify when + * the receiving of data should be stopped. + */ def isStopped(): Boolean = { executor.isReceiverStopped() } - /** Get unique identifier of this receiver. */ + /** + * Get the unique identifier the receiver input stream that this + * receiver is associated with. + */ def streamId = id /* diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java new file mode 100644 index 0000000000000..1b0787fe69dec --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -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.streaming; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.junit.Assert.*; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.receiver.Receiver; +import org.apache.spark.api.java.function.Function; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.io.Serializable; +import java.net.ConnectException; +import java.net.Socket; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class JavaReceiverAPISuite implements Serializable { + + @Before + public void setUp() { + System.clearProperty("spark.streaming.clock"); + } + + @After + public void tearDown() { + System.clearProperty("spark.streaming.clock"); + } + + @Test + public void testReceiver() throws InterruptedException { + TestServer server = new TestServer(0); + server.start(); + + final AtomicLong dataCounter = new AtomicLong(0); + + try { + JavaStreamingContext ssc = new JavaStreamingContext("local[2]", "test", new Duration(200)); + JavaReceiverInputDStream input = + ssc.receiverStream(new JavaSocketReceiver("localhost", server.port())); + JavaDStream mapped = input.map(new Function() { + @Override + public String call(String v1) throws Exception { + return v1 + "."; + } + }); + mapped.foreachRDD(new Function, Void>() { + @Override + public Void call(JavaRDD rdd) throws Exception { + long count = rdd.count(); + dataCounter.addAndGet(count); + return null; + } + }); + + ssc.start(); + long startTime = System.currentTimeMillis(); + long timeout = 10000; + + Thread.sleep(200); + for (int i = 0; i < 6; i++) { + server.send("" + i + "\n"); // \n to make sure these are separate lines + Thread.sleep(100); + } + while (dataCounter.get() == 0 && System.currentTimeMillis() - startTime < timeout) { + Thread.sleep(100); + } + ssc.stop(); + assertTrue(dataCounter.get() > 0); + } finally { + server.stop(); + } + } +} + +class JavaSocketReceiver extends Receiver { + + String host = null; + int port = -1; + + public JavaSocketReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK()); + host = host_; + port = port_; + } + + @Override + public void onStart() { + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + @Override + public void onStop() { + } + + private void receive() { + Socket socket = null; + try { + socket = new Socket(host, port); + BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream())); + String userInput; + while ((userInput = in.readLine()) != null) { + store(userInput); + } + in.close(); + socket.close(); + } catch(ConnectException ce) { + ce.printStackTrace(); + restart("Could not connect", ce); + } catch(Throwable t) { + t.printStackTrace(); + restart("Error receiving data", t); + } + } +} + diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index 33f6df8f88177..c0ea0491c313d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -26,6 +26,7 @@ import org.apache.spark.streaming._ import java.util.ArrayList import collection.JavaConversions._ import org.apache.spark.api.java.JavaRDDLike +import org.apache.spark.streaming.dstream.DStream /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { @@ -51,8 +52,7 @@ trait JavaTestBase extends TestSuiteBase { * [[org.apache.spark.streaming.TestOutputStream]]. **/ def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = - { + dstream: JavaDStreamLike[T, This, R]) = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStreamWithPartitions(dstream.dstream) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index b55b7834c90c1..3fa254065cc44 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -49,7 +49,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val networkStream = ssc.socketTextStream( + "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 3e2b25af84098..ee0bc8b7d6a71 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -165,7 +165,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 TestReceiver.counter.set(1) - val input = ssc.networkStream(new TestReceiver) + val input = ssc.receiverStream(new TestReceiver) input.count.foreachRDD(rdd => { val count = rdd.first() runningCount += count.toInt diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 4f63fd37822cb..8036f77c973ae 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -155,6 +155,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def afterFunction() { // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") + System.clearProperty("spark.streaming.clock") } before(beforeFunction) diff --git a/tools/pom.xml b/tools/pom.xml index ae2ba64e07c21..1875c497bc61c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,20 +30,6 @@ Spark Project Tools http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index d0aeaceb0d23c..e076ca1d44b97 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn-alpha_2.10 jar diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 00c7649e68e13..8226207de42b8 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -169,8 +169,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f2be8217a2f8a..27a518ccda459 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -263,9 +263,13 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + // Allow users to specify some environment variables. + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) + + // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. + env("SPARK_YARN_USER_ENV") = userEnvs + } env } @@ -322,6 +326,12 @@ trait ClientBase extends Logging { JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } + // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility: + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + sparkConf.set("spark.executor.extraJavaOptions", opts) + sparkConf.set("spark.driver.extraJavaOptions", opts) + } + // TODO: it might be nicer to pass these as an internal environment variable rather than // as Java options, due to complications with string parsing of nested quotes. if (args.amClass == classOf[ExecutorLauncher].getName) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 7d07f6f68046a..96f8aa93394f5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -71,8 +71,8 @@ trait ExecutorRunnableUtil extends Logging { /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont - // want to mess with it. + // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions, + // so we dont want to mess with it. // In our expts, using (default) throughput collector has severe perf ramnifications in // multi-tennent machines // The options are based on diff --git a/yarn/pom.xml b/yarn/pom.xml index 3342cb65edcd1..02f36627431b9 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -70,15 +70,6 @@ alpha - - - - - org.apache.avro - avro - - diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index e7915d12aef63..0780f251b595c 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,20 +24,6 @@ ../pom.xml - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark spark-yarn_2.10 jar diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2e2fb5d4fa787..1b6bfb42a5c1c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -171,8 +171,10 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master yarn\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a " + + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") + } // Set an env variable indicating we are running in YARN mode. // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -