From 077213bae09737ccb904f07b2766d43bb0734477 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 15 Aug 2014 17:04:15 -0700 Subject: [PATCH 001/192] [SPARK-3046] use executor's class loader as the default serializer classloader The serializer is not always used in an executor thread (e.g. connection manager, broadcast), in which case the classloader might not have the user jar set, leading to corruption in deserialization. https://issues.apache.org/jira/browse/SPARK-3046 https://issues.apache.org/jira/browse/SPARK-2878 Author: Reynold Xin Closes #1972 from rxin/kryoBug and squashes the following commits: c1c7bf0 [Reynold Xin] Made change to JavaSerializer. 7204c33 [Reynold Xin] Added imports back. d879e67 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer class loader. (cherry picked from commit cc3648774e9a744850107bb187f2828d447e0a48) Signed-off-by: Reynold Xin --- .../org/apache/spark/executor/Executor.scala | 3 + .../spark/serializer/JavaSerializer.scala | 9 ++- .../spark/serializer/KryoSerializer.scala | 9 ++- .../apache/spark/serializer/Serializer.scala | 17 +++++ .../KryoSerializerDistributedSuite.scala | 71 +++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 23 +++++- 6 files changed, 128 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index eac1f2326a29d..fb3f7bd54bbfa 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -99,6 +99,9 @@ private[spark] class Executor( private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + // Set the classloader for serializer + env.serializer.setDefaultClassLoader(urlClassLoader) + // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 34bc3124097bb..af33a2f2ca3e1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -63,7 +63,9 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { +private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader) + extends SerializerInstance { + def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -109,7 +111,10 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) + override def newInstance(): SerializerInstance = { + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) + new JavaSerializerInstance(counterReset, classLoader) + } override def writeExternal(out: ObjectOutput) { out.writeInt(counterReset) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 85944eabcfefc..99682220b4ab5 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -61,7 +61,9 @@ class KryoSerializer(conf: SparkConf) val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() kryo.setRegistrationRequired(registrationRequired) - val classLoader = Thread.currentThread.getContextClassLoader + + val oldClassLoader = Thread.currentThread.getContextClassLoader + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. @@ -84,10 +86,15 @@ class KryoSerializer(conf: SparkConf) try { val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] + + // Use the default classloader when calling the user registrator. + Thread.currentThread.setContextClassLoader(classLoader) reg.registerClasses(kryo) } catch { case e: Exception => throw new SparkException(s"Failed to invoke $regCls", e) + } finally { + Thread.currentThread.setContextClassLoader(oldClassLoader) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index f2f5cea469c61..e674438c8176c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -44,6 +44,23 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} */ @DeveloperApi trait Serializer { + + /** + * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should + * make sure it is using this when set. + */ + @volatile protected var defaultClassLoader: Option[ClassLoader] = None + + /** + * Sets a class loader for the serializer to use in deserialization. + * + * @return this Serializer object + */ + def setDefaultClassLoader(classLoader: ClassLoader): Serializer = { + defaultClassLoader = Some(classLoader) + this + } + def newInstance(): SerializerInstance } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala new file mode 100644 index 0000000000000..11e8c9c4cb37f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -0,0 +1,71 @@ +/* + * 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.serializer + +import org.apache.spark.util.Utils + +import com.esotericsoftware.kryo.Kryo +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, TestUtils} +import org.apache.spark.SparkContext._ +import org.apache.spark.serializer.KryoDistributedTest._ + +class KryoSerializerDistributedSuite extends FunSuite { + + test("kryo objects are serialised consistently in different processes") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + conf.set("spark.task.maxFailures", "1") + + val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) + conf.setJars(List(jar.getPath)) + + val sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + val original = Thread.currentThread.getContextClassLoader + val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + SparkEnv.get.serializer.setDefaultClassLoader(loader) + + val cachedRDD = sc.parallelize((0 until 10).map((_, new MyCustomClass)), 3).cache() + + // Randomly mix the keys so that the join below will require a shuffle with each partition + // sending data to multiple other partitions. + val shuffledRDD = cachedRDD.map { case (i, o) => (i * i * i - 10 * i * i, o)} + + // Join the two RDDs, and force evaluation + assert(shuffledRDD.join(cachedRDD).collect().size == 1) + + LocalSparkContext.stop(sc) + } +} + +object KryoDistributedTest { + class MyCustomClass + + class AppJarRegistrator extends KryoRegistrator { + override def registerClasses(k: Kryo) { + val classLoader = Thread.currentThread.getContextClassLoader + k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader)) + } + } + + object AppJarRegistrator { + val customClassName = "KryoSerializerDistributedSuiteCustomClass" + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 3bf9efebb39d2..a579fd50bd9e4 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.SharedSparkContext +import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { @@ -217,8 +217,29 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) assert(thrown.getMessage.contains("Failed to invoke this.class.does.not.exist")) } + + test("default class loader can be set by a different thread") { + val ser = new KryoSerializer(new SparkConf) + + // First serialize the object + val serInstance = ser.newInstance() + val bytes = serInstance.serialize(new ClassLoaderTestingObject) + + // Deserialize the object to make sure normal deserialization works + serInstance.deserialize[ClassLoaderTestingObject](bytes) + + // Set a special, broken ClassLoader and make sure we get an exception on deserialization + ser.setDefaultClassLoader(new ClassLoader() { + override def loadClass(name: String) = throw new UnsupportedOperationException + }) + intercept[UnsupportedOperationException] { + ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) + } + } } +class ClassLoaderTestingObject + class KryoSerializerResizableOutputSuite extends FunSuite { import org.apache.spark.SparkConf import org.apache.spark.SparkContext From c085011cac4df1bf4cbaef00a8b921ace6e3123b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 15 Aug 2014 21:04:29 -0700 Subject: [PATCH 002/192] [SPARK-3078][MLLIB] Make LRWithLBFGS API consistent with others Should ask users to set parameters through the optimizer. dbtsai Author: Xiangrui Meng Closes #1973 from mengxr/lr-lbfgs and squashes the following commits: e3efbb1 [Xiangrui Meng] fix tests 21b3579 [Xiangrui Meng] fix method name 641eea4 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into lr-lbfgs 456ab7c [Xiangrui Meng] update LRWithLBFGS (cherry picked from commit 5d25c0b74f6397d78164b96afb8b8cbb1b15cfbd) Signed-off-by: Xiangrui Meng --- .../examples/mllib/BinaryClassification.scala | 8 ++-- .../classification/LogisticRegression.scala | 40 +++---------------- .../spark/mllib/optimization/LBFGS.scala | 9 +++++ .../LogisticRegressionSuite.scala | 5 ++- .../spark/mllib/optimization/LBFGSSuite.scala | 24 +++++------ 5 files changed, 33 insertions(+), 53 deletions(-) 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 index 56b02b65d8724..a6f78d2441db1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -21,7 +21,7 @@ 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.classification.{LogisticRegressionWithLBFGS, SVMWithSGD} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} @@ -66,7 +66,8 @@ object BinaryClassification { .text("number of iterations") .action((x, c) => c.copy(numIterations = x)) opt[Double]("stepSize") - .text(s"initial step size, default: ${defaultParams.stepSize}") + .text("initial step size (ignored by logistic regression), " + + s"default: ${defaultParams.stepSize}") .action((x, c) => c.copy(stepSize = x)) opt[String]("algorithm") .text(s"algorithm (${Algorithm.values.mkString(",")}), " + @@ -125,10 +126,9 @@ object BinaryClassification { val model = params.algorithm match { case LR => - val algorithm = new LogisticRegressionWithSGD() + val algorithm = new LogisticRegressionWithLBFGS() algorithm.optimizer .setNumIterations(params.numIterations) - .setStepSize(params.stepSize) .setUpdater(updater) .setRegParam(params.regParam) algorithm.run(training).clearThreshold() 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 6790c86f651b4..486bdbfa9cb47 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 @@ -73,6 +73,8 @@ class LogisticRegressionModel ( /** * Train a classification model for Logistic Regression using Stochastic Gradient Descent. * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ class LogisticRegressionWithSGD private ( private var stepSize: Double, @@ -191,51 +193,19 @@ object LogisticRegressionWithSGD { /** * Train a classification model for Logistic Regression using Limited-memory BFGS. + * Standard feature scaling and L2 regularization are used by default. * NOTE: Labels used in Logistic Regression should be {0, 1} */ -class LogisticRegressionWithLBFGS private ( - private var convergenceTol: Double, - private var maxNumIterations: Int, - private var regParam: Double) +class LogisticRegressionWithLBFGS extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1E-4, 100, 0.0) - this.setFeatureScaling(true) - private val gradient = new LogisticGradient() - private val updater = new SimpleUpdater() - // Have to return new LBFGS object every time since users can reset the parameters anytime. - override def optimizer = new LBFGS(gradient, updater) - .setNumCorrections(10) - .setConvergenceTol(convergenceTol) - .setMaxNumIterations(maxNumIterations) - .setRegParam(regParam) + override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) override protected val validators = List(DataValidators.binaryLabelValidator) - /** - * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. - * Smaller value will lead to higher accuracy with the cost of more iterations. - */ - def setConvergenceTol(convergenceTol: Double): this.type = { - this.convergenceTol = convergenceTol - this - } - - /** - * Set the maximal number of iterations for L-BFGS. Default 100. - */ - def setNumIterations(numIterations: Int): this.type = { - this.maxNumIterations = numIterations - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 033fe44f34f3c..d16d0daf08565 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -69,8 +69,17 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the maximal number of iterations for L-BFGS. Default 100. + * @deprecated use [[LBFGS#setNumIterations]] instead */ + @deprecated("use setNumIterations instead", "1.1.0") def setMaxNumIterations(iters: Int): this.type = { + this.setNumIterations(iters) + } + + /** + * Set the maximal number of iterations for L-BFGS. Default 100. + */ + def setNumIterations(iters: Int): this.type = { this.maxNumIterations = iters this } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index bc05b2046878f..862178694a50e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -272,8 +272,9 @@ class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkCont }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = - (new LogisticRegressionWithLBFGS().setIntercept(true).setNumIterations(2)).run(points) + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + lr.optimizer.setNumIterations(2) + val model = lr.run(points) val predictions = model.predict(points.map(_.features)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 5f4c24115ac80..ccba004baa007 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -55,7 +55,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { val initialWeightsWithIntercept = Vectors.dense(1.0 +: initialWeights.toArray) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (_, loss) = LBFGS.runLBFGS( dataRDD, @@ -63,7 +63,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { simpleUpdater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -99,7 +99,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (weightLBFGS, lossLBFGS) = LBFGS.runLBFGS( dataRDD, @@ -107,7 +107,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -140,10 +140,10 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { /** * For the first run, we set the convergenceTol to 0.0, so that the algorithm will - * run up to the maxNumIterations which is 8 here. + * run up to the numIterations which is 8 here. */ val initialWeightsWithIntercept = Vectors.dense(0.0, 0.0) - val maxNumIterations = 8 + val numIterations = 8 var convergenceTol = 0.0 val (_, lossLBFGS1) = LBFGS.runLBFGS( @@ -152,7 +152,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -167,7 +167,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -182,7 +182,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -200,12 +200,12 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val lbfgsOptimizer = new LBFGS(gradient, squaredL2Updater) .setNumCorrections(numCorrections) .setConvergenceTol(convergenceTol) - .setMaxNumIterations(maxNumIterations) + .setNumIterations(numIterations) .setRegParam(regParam) val weightLBFGS = lbfgsOptimizer.optimize(dataRDD, initialWeightsWithIntercept) @@ -241,7 +241,7 @@ class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) .setNumCorrections(1) .setConvergenceTol(1e-12) - .setMaxNumIterations(1) + .setNumIterations(1) .setRegParam(1.0) val random = new Random(0) // If we serialize data directly in the task closure, the size of the serialized task would be From ce06d7f45bc551f6121c382b0833e01b8a83f636 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 15 Aug 2014 21:07:55 -0700 Subject: [PATCH 003/192] [SPARK-3001][MLLIB] Improve Spearman's correlation The current implementation requires sorting individual columns, which could be done with a global sort. result on a 32-node cluster: m | n | prev | this ---|---|-------|----- 1000000 | 50 | 55s | 9s 10000000 | 50 | 97s | 76s 1000000 | 100 | 119s | 15s Author: Xiangrui Meng Closes #1917 from mengxr/spearman and squashes the following commits: 4d5d262 [Xiangrui Meng] remove unused import 85c48de [Xiangrui Meng] minor updates a048d0c [Xiangrui Meng] remove cache and set a limit to cachedIds b98bb18 [Xiangrui Meng] add comments 0846e07 [Xiangrui Meng] first version (cherry picked from commit 2e069ca6560bf7ab07bd019f9530b42f4fe45014) Signed-off-by: Xiangrui Meng --- .../correlation/SpearmanCorrelation.scala | 120 ++++++------------ 1 file changed, 42 insertions(+), 78 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index 9bd0c2cd05de4..4a6c677f06d28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.stat.correlation import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, HashPartitioner} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{DenseVector, Matrix, Vector} -import org.apache.spark.rdd.{CoGroupedRDD, RDD} +import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors} +import org.apache.spark.rdd.RDD /** * Compute Spearman's correlation for two RDDs of the type RDD[Double] or the correlation matrix @@ -43,87 +43,51 @@ private[stat] object SpearmanCorrelation extends Correlation with Logging { /** * Compute Spearman's correlation matrix S, for the input matrix, where S(i, j) is the * correlation between column i and j. - * - * Input RDD[Vector] should be cached or checkpointed if possible since it would be split into - * numCol RDD[Double]s, each of which sorted, and the joined back into a single RDD[Vector]. */ override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { - val indexed = X.zipWithUniqueId() - - val numCols = X.first.size - if (numCols > 50) { - logWarning("Computing the Spearman correlation matrix can be slow for large RDDs with more" - + " than 50 columns.") - } - val ranks = new Array[RDD[(Long, Double)]](numCols) - - // Note: we use a for loop here instead of a while loop with a single index variable - // to avoid race condition caused by closure serialization - for (k <- 0 until numCols) { - val column = indexed.map { case (vector, index) => (vector(k), index) } - ranks(k) = getRanks(column) + // ((columnIndex, value), rowUid) + val colBased = X.zipWithUniqueId().flatMap { case (vec, uid) => + vec.toArray.view.zipWithIndex.map { case (v, j) => + ((j, v), uid) + } } - - val ranksMat: RDD[Vector] = makeRankMatrix(ranks, X) - PearsonCorrelation.computeCorrelationMatrix(ranksMat) - } - - /** - * Compute the ranks for elements in the input RDD, using the average method for ties. - * - * With the average method, elements with the same value receive the same rank that's computed - * by taking the average of their positions in the sorted list. - * e.g. ranks([2, 1, 0, 2]) = [2.5, 1.0, 0.0, 2.5] - * Note that positions here are 0-indexed, instead of the 1-indexed as in the definition for - * ranks in the standard definition for Spearman's correlation. This does not affect the final - * results and is slightly more performant. - * - * @param indexed RDD[(Double, Long)] containing pairs of the format (originalValue, uniqueId) - * @return RDD[(Long, Double)] containing pairs of the format (uniqueId, rank), where uniqueId is - * copied from the input RDD. - */ - private def getRanks(indexed: RDD[(Double, Long)]): RDD[(Long, Double)] = { - // Get elements' positions in the sorted list for computing average rank for duplicate values - val sorted = indexed.sortByKey().zipWithIndex() - - val ranks: RDD[(Long, Double)] = sorted.mapPartitions { iter => - // add an extra element to signify the end of the list so that flatMap can flush the last - // batch of duplicates - val end = -1L - val padded = iter ++ Iterator[((Double, Long), Long)](((Double.NaN, end), end)) - val firstEntry = padded.next() - var lastVal = firstEntry._1._1 - var firstRank = firstEntry._2.toDouble - val idBuffer = ArrayBuffer(firstEntry._1._2) - padded.flatMap { case ((v, id), rank) => - if (v == lastVal && id != end) { - idBuffer += id - Iterator.empty - } else { - val entries = if (idBuffer.size == 1) { - Iterator((idBuffer(0), firstRank)) - } else { - val averageRank = firstRank + (idBuffer.size - 1.0) / 2.0 - idBuffer.map(id => (id, averageRank)) - } - lastVal = v - firstRank = rank - idBuffer.clear() - idBuffer += id - entries + // global sort by (columnIndex, value) + val sorted = colBased.sortByKey() + // assign global ranks (using average ranks for tied values) + val globalRanks = sorted.zipWithIndex().mapPartitions { iter => + var preCol = -1 + var preVal = Double.NaN + var startRank = -1.0 + var cachedUids = ArrayBuffer.empty[Long] + val flush: () => Iterable[(Long, (Int, Double))] = () => { + val averageRank = startRank + (cachedUids.size - 1) / 2.0 + val output = cachedUids.map { uid => + (uid, (preCol, averageRank)) } + cachedUids.clear() + output } + iter.flatMap { case (((j, v), uid), rank) => + // If we see a new value or cachedUids is too big, we flush ids with their average rank. + if (j != preCol || v != preVal || cachedUids.size >= 10000000) { + val output = flush() + preCol = j + preVal = v + startRank = rank + cachedUids += uid + output + } else { + cachedUids += uid + Iterator.empty + } + } ++ flush() } - ranks - } - - private def makeRankMatrix(ranks: Array[RDD[(Long, Double)]], input: RDD[Vector]): RDD[Vector] = { - val partitioner = new HashPartitioner(input.partitions.size) - val cogrouped = new CoGroupedRDD[Long](ranks, partitioner) - cogrouped.map { - case (_, values: Array[Iterable[_]]) => - val doubles = values.asInstanceOf[Array[Iterable[Double]]] - new DenseVector(doubles.flatten.toArray) + // Replace values in the input matrix by their ranks compared with values in the same column. + // Note that shifting all ranks in a column by a constant value doesn't affect result. + val groupedRanks = globalRanks.groupByKey().map { case (uid, iter) => + // sort by column index and then convert values to a vector + Vectors.dense(iter.toSeq.sortBy(_._1).map(_._2).toArray) } + PearsonCorrelation.computeCorrelationMatrix(groupedRanks) } } From 2541537217fd3f73e494c98d4c5e379723fe0199 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Aug 2014 22:55:32 -0700 Subject: [PATCH 004/192] [SPARK-3015] Block on cleaning tasks to prevent Akka timeouts More detail on the issue is described in [SPARK-3015](https://issues.apache.org/jira/browse/SPARK-3015), but the TLDR is if we send too many blocking Akka messages that are dependent on each other in quick successions, then we end up causing a few of these messages to time out and ultimately kill the executors. As of #1498, we broadcast each RDD whether or not it is persisted. This means if we create many RDDs (each of which becomes a broadcast) and the driver performs a GC that cleans up all of these broadcast blocks, then we end up sending many `RemoveBroadcast` messages in parallel and trigger the chain of blocking messages at high frequencies. We do not know of the Akka-level root cause yet, so this is intended to be a temporary solution until we identify the real issue. I have done some preliminary testing of enabling blocking and observed that the queue length remains quite low (< 1000) even under very intensive workloads. In the long run, we should do something more sophisticated to allow a limited degree of parallelism through batching clean up tasks or processing them in a sliding window. In the longer run, we should clean up the whole `BlockManager*` message passing interface to avoid unnecessarily awaiting on futures created from Akka asks. tdas pwendell mengxr Author: Andrew Or Closes #1931 from andrewor14/reference-blocking and squashes the following commits: d0f7195 [Andrew Or] Merge branch 'master' of github.com:apache/spark into reference-blocking ce9daf5 [Andrew Or] Remove logic for logging queue length 111192a [Andrew Or] Add missing space in log message (minor) a183b83 [Andrew Or] Switch order of code blocks (minor) 9fd1fe6 [Andrew Or] Remove outdated log 104b366 [Andrew Or] Use the actual reference queue length 0b7e768 [Andrew Or] Block on cleaning tasks by default + log error on queue full (cherry picked from commit c9da466edb83e45a159ccc17c68856a511b9e8b7) Signed-off-by: Patrick Wendell --- .../main/scala/org/apache/spark/ContextCleaner.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index bf3c3a6ceb5ef..3848734d6f639 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -66,10 +66,15 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Whether the cleaning thread will block on cleanup tasks. - * This is set to true only for tests. + * + * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary + * workaround for the issue, which is ultimately caused by the way the BlockManager actors + * issue inter-dependent blocking Akka messages to each other at high frequencies. This happens, + * for instance, when the driver performs a GC and cleans up all broadcast blocks that are no + * longer in scope. */ private val blockOnCleanupTasks = sc.conf.getBoolean( - "spark.cleaner.referenceTracking.blocking", false) + "spark.cleaner.referenceTracking.blocking", true) @volatile private var stopped = false @@ -174,9 +179,6 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - - // Used for testing. These methods explicitly blocks until cleanup is completed - // to ensure that more reliable testing. } private object ContextCleaner { From fcf30cdc558aff4c615e4d8f0bbe30e39a0448e4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 15 Aug 2014 23:12:34 -0700 Subject: [PATCH 005/192] [SPARK-3045] Make Serializer interface Java friendly Author: Reynold Xin Closes #1948 from rxin/kryo and squashes the following commits: a3a80d8 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer classloader 3d13277 [Reynold Xin] Reverted that in TestJavaSerializerImpl too. 196f3dc [Reynold Xin] Ok one more commit to revert the classloader change. c49b50c [Reynold Xin] Removed JavaSerializer change. afbf37d [Reynold Xin] Moved the test case also. a2e693e [Reynold Xin] Removed the Kryo bug fix from this pull request. c81bd6c [Reynold Xin] Use defaultClassLoader when executing user specified custom registrator. 68f261e [Reynold Xin] Added license check excludes. 0c28179 [Reynold Xin] [SPARK-3045] Make Serializer interface Java friendly [SPARK-3046] Set executor's class loader as the default serializer class loader (cherry picked from commit a83c7723bf7a90dc6cd5dde98a179303b7542020) Signed-off-by: Reynold Xin --- .../spark/serializer/JavaSerializer.scala | 15 +-- .../spark/serializer/KryoSerializer.scala | 32 +++---- .../apache/spark/serializer/Serializer.scala | 25 ++--- .../apache/spark/serializer/package-info.java | 2 +- .../serializer/TestJavaSerializerImpl.java | 95 +++++++++++++++++++ .../KryoSerializerResizableOutputSuite.scala | 52 ++++++++++ .../serializer/KryoSerializerSuite.scala | 34 +------ project/MimaExcludes.scala | 11 +++ 8 files changed, 193 insertions(+), 73 deletions(-) create mode 100644 core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index af33a2f2ca3e1..554a33ce7f1a6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -63,10 +63,11 @@ extends DeserializationStream { def close() { objIn.close() } } + private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader) extends SerializerInstance { - def serialize[T: ClassTag](t: T): ByteBuffer = { + override def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -74,23 +75,23 @@ private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoade ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) - in.readObject().asInstanceOf[T] + in.readObject() } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) - in.readObject().asInstanceOf[T] + in.readObject() } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new JavaSerializationStream(s, counterReset) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 99682220b4ab5..87ef9bb0b43c6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -91,7 +91,7 @@ class KryoSerializer(conf: SparkConf) Thread.currentThread.setContextClassLoader(classLoader) reg.registerClasses(kryo) } catch { - case e: Exception => + case e: Exception => throw new SparkException(s"Failed to invoke $regCls", e) } finally { Thread.currentThread.setContextClassLoader(oldClassLoader) @@ -106,7 +106,7 @@ class KryoSerializer(conf: SparkConf) kryo } - def newInstance(): SerializerInstance = { + override def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } } @@ -115,20 +115,20 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T: ClassTag](t: T): SerializationStream = { + override def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } - def flush() { output.flush() } - def close() { output.close() } + override def flush() { output.flush() } + override def close() { output.close() } } private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - val input = new KryoInput(inStream) + private val input = new KryoInput(inStream) - def readObject[T: ClassTag](): T = { + override def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -138,31 +138,31 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } } - def close() { + override def close() { // Kryo's Input automatically closes the input stream it is using. input.close() } } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - val kryo = ks.newKryo() + private val kryo = ks.newKryo() // Make these lazy vals to avoid creating a buffer unless we use them - lazy val output = ks.newKryoOutput() - lazy val input = new KryoInput() + private lazy val output = ks.newKryoOutput() + private lazy val input = new KryoInput() - def serialize[T: ClassTag](t: T): ByteBuffer = { + override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) @@ -171,11 +171,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ obj } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new KryoSerializationStream(kryo, s) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new KryoDeserializationStream(kryo, s) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index e674438c8176c..a9144cdd97b8c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} * They are intended to be used to serialize/de-serialize data within a single Spark application. */ @DeveloperApi -trait Serializer { +abstract class Serializer { /** * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should @@ -61,10 +61,12 @@ trait Serializer { this } + /** Creates a new [[SerializerInstance]]. */ def newInstance(): SerializerInstance } +@DeveloperApi object Serializer { def getSerializer(serializer: Serializer): Serializer = { if (serializer == null) SparkEnv.get.serializer else serializer @@ -81,7 +83,7 @@ object Serializer { * An instance of a serializer, for use by one thread at a time. */ @DeveloperApi -trait SerializerInstance { +abstract class SerializerInstance { def serialize[T: ClassTag](t: T): ByteBuffer def deserialize[T: ClassTag](bytes: ByteBuffer): T @@ -91,21 +93,6 @@ trait SerializerInstance { def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - - def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { - // Default implementation uses serializeStream - val stream = new ByteArrayOutputStream() - serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.wrap(stream.toByteArray) - buffer.flip() - buffer - } - - def deserializeMany(buffer: ByteBuffer): Iterator[Any] = { - // Default implementation uses deserializeStream - buffer.rewind() - deserializeStream(new ByteBufferInputStream(buffer)).asIterator - } } /** @@ -113,7 +100,7 @@ trait SerializerInstance { * A stream for writing serialized objects. */ @DeveloperApi -trait SerializationStream { +abstract class SerializationStream { def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit @@ -132,7 +119,7 @@ trait SerializationStream { * A stream for reading serialized objects. */ @DeveloperApi -trait DeserializationStream { +abstract class DeserializationStream { def readObject[T: ClassTag](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java index 4c0b73ab36a00..207c6e02e4293 100644 --- a/core/src/main/scala/org/apache/spark/serializer/package-info.java +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -18,4 +18,4 @@ /** * Pluggable serializers for RDD and shuffle data. */ -package org.apache.spark.serializer; \ No newline at end of file +package org.apache.spark.serializer; diff --git a/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java new file mode 100644 index 0000000000000..3d50ab4fabe42 --- /dev/null +++ b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java @@ -0,0 +1,95 @@ +/* + * 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.serializer; + +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import scala.Option; +import scala.reflect.ClassTag; + + +/** + * A simple Serializer implementation to make sure the API is Java-friendly. + */ +class TestJavaSerializerImpl extends Serializer { + + @Override + public SerializerInstance newInstance() { + return null; + } + + static class SerializerInstanceImpl extends SerializerInstance { + @Override + public ByteBuffer serialize(T t, ClassTag evidence$1) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassLoader loader, ClassTag evidence$1) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassTag evidence$1) { + return null; + } + + @Override + public SerializationStream serializeStream(OutputStream s) { + return null; + } + + @Override + public DeserializationStream deserializeStream(InputStream s) { + return null; + } + } + + static class SerializationStreamImpl extends SerializationStream { + + @Override + public SerializationStream writeObject(T t, ClassTag evidence$1) { + return null; + } + + @Override + public void flush() { + + } + + @Override + public void close() { + + } + } + + static class DeserializationStreamImpl extends DeserializationStream { + + @Override + public T readObject(ClassTag evidence$1) { + return null; + } + + @Override + public void close() { + + } + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala new file mode 100644 index 0000000000000..967c9e9899c9d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -0,0 +1,52 @@ +/* + * 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.serializer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.LocalSparkContext +import org.apache.spark.SparkException + + +class KryoSerializerResizableOutputSuite extends FunSuite { + + // trial and error showed this will not serialize with 1mb buffer + val x = (1 to 400000).toArray + + test("kryo without resizable output buffer should fail on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "1") + val sc = new SparkContext("local", "test", conf) + intercept[SparkException](sc.parallelize(x).collect()) + LocalSparkContext.stop(sc) + } + + test("kryo with resizable output buffer should succeed on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "2") + val sc = new SparkContext("local", "test", conf) + assert(sc.parallelize(x).collect() === x) + LocalSparkContext.stop(sc) + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index a579fd50bd9e4..e1e35b688d581 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ + class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) @@ -207,7 +208,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x assert(10 + control.sum === result) } - + test("kryo with nonexistent custom registrator should fail") { import org.apache.spark.{SparkConf, SparkException} @@ -238,39 +239,12 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } } -class ClassLoaderTestingObject - -class KryoSerializerResizableOutputSuite extends FunSuite { - import org.apache.spark.SparkConf - import org.apache.spark.SparkContext - import org.apache.spark.LocalSparkContext - import org.apache.spark.SparkException - - // trial and error showed this will not serialize with 1mb buffer - val x = (1 to 400000).toArray - test("kryo without resizable output buffer should fail on large array") { - val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "1") - val sc = new SparkContext("local", "test", conf) - intercept[SparkException](sc.parallelize(x).collect) - LocalSparkContext.stop(sc) - } +class ClassLoaderTestingObject - test("kryo with resizable output buffer should succeed on large array") { - val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "2") - val sc = new SparkContext("local", "test", conf) - assert(sc.parallelize(x).collect === x) - LocalSparkContext.stop(sc) - } -} object KryoTest { + case class CaseClass(i: Int, s: String) {} class ClassWithNoArgConstructor { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1e3c760b845de..bbe68b29d2d8e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -61,6 +61,17 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry") ) ++ + Seq( + // Serializer interface change. See SPARK-3045. + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.DeserializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.Serializer"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializerInstance") + )++ Seq( // Renamed putValues -> putArray + putIterator ProblemFilters.exclude[MissingMethodProblem]( From 0e0ec2eeb1eab1cb6dabbaa60d30242d0d7e292f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 16 Aug 2014 00:04:55 -0700 Subject: [PATCH 006/192] [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager This is intended to fix SPARK-2977. Before, there was an implicit ordering dependency where we needed to know the ShuffleManager implementation before creating the ShuffleBlockManager. This patch makes that dependency explicit by adding ShuffleManager to a bunch of constructors. I think it's a little odd for BlockManager to take a ShuffleManager only to pass it to ShuffleBlockManager without using it itself; there's an opportunity to clean this up later if we sever the circular dependencies between BlockManager and other components and pass those components to BlockManager's constructor. Author: Josh Rosen Closes #1976 from JoshRosen/SPARK-2977 and squashes the following commits: a9cd1e1 [Josh Rosen] [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager. (cherry picked from commit 20fcf3d0b72f3707dc1ed95d453f570fabdefd16) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/SparkEnv.scala | 22 +++++++++---------- .../apache/spark/storage/BlockManager.scala | 11 ++++++---- .../spark/storage/ShuffleBlockManager.scala | 7 +++--- .../apache/spark/storage/ThreadingTest.scala | 3 ++- .../spark/storage/BlockManagerSuite.scala | 12 +++++----- .../spark/storage/DiskBlockManagerSuite.scala | 8 +++++-- 6 files changed, 37 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 22d8d1cb1ddcf..fc36e37c53f5e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -210,12 +210,22 @@ object SparkEnv extends Logging { "MapOutputTracker", new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + // Let the user specify short names for shuffle managers + val shortShuffleMgrNames = Map( + "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", + "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") + val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) + val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) + + val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker) + serializer, conf, securityManager, mapOutputTracker, shuffleManager) val connectionManager = blockManager.connectionManager @@ -250,16 +260,6 @@ object SparkEnv extends Logging { "." } - // Let the user specify short names for shuffle managers - val shortShuffleMgrNames = Map( - "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", - "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") - val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") - val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) - val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) - - val shuffleMemoryManager = new ShuffleMemoryManager(conf) - // Warn about deprecated spark.cache.class property if (conf.contains("spark.cache.class")) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + 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 e8bbd298c631a..e4c3d58905e7f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -33,6 +33,7 @@ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ private[spark] sealed trait BlockValues @@ -57,11 +58,12 @@ private[spark] class BlockManager( maxMemory: Long, val conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) extends Logging { private val port = conf.getInt("spark.blockManager.port", 0) - val shuffleBlockManager = new ShuffleBlockManager(this) + val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) val connectionManager = @@ -142,9 +144,10 @@ private[spark] class BlockManager( serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) = { + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker) + conf, securityManager, mapOutputTracker, shuffleManager) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 3565719b54545..b8f5d3a5b02aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.Logging import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} @@ -62,7 +63,8 @@ private[spark] trait ShuffleWriterGroup { */ // TODO: Factor this into a separate class for each ShuffleManager implementation private[spark] -class ShuffleBlockManager(blockManager: BlockManager) extends Logging { +class ShuffleBlockManager(blockManager: BlockManager, + shuffleManager: ShuffleManager) extends Logging { def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. @@ -71,8 +73,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { conf.getBoolean("spark.shuffle.consolidateFiles", false) // Are we using sort-based shuffle? - val sortBasedShuffle = - conf.get("spark.shuffle.manager", "") == classOf[SortShuffleManager].getName + val sortBasedShuffle = shuffleManager.isInstanceOf[SortShuffleManager] private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 75c2e09a6bbb8..aa83ea90ee9ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.util.concurrent.ArrayBlockingQueue import akka.actor._ +import org.apache.spark.shuffle.hash.HashShuffleManager import util.Random import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} @@ -101,7 +102,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) 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 94bb2c445d2e9..20bac66105a69 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit import akka.actor._ import akka.pattern.ask import akka.util.Timeout +import org.apache.spark.shuffle.hash.HashShuffleManager import org.mockito.invocation.InvocationOnMock import org.mockito.Matchers.any @@ -61,6 +62,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -71,8 +73,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { - new BlockManager( - name, actorSystem, master, serializer, maxMem, conf, securityMgr, mapOutputTracker) + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, + mapOutputTracker, shuffleManager) } before { @@ -791,7 +793,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) // The put should fail since a1 is not serializable. class UnserializableClass @@ -1007,7 +1009,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) val worker = spy(new BlockManagerWorker(store)) val connManagerId = mock(classOf[ConnectionManagerId]) @@ -1054,7 +1056,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) val worker = spy(new BlockManagerWorker(store)) val connManagerId = mock(classOf[ConnectionManagerId]) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index b8299e2ea187f..777579bc570db 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.shuffle.hash.HashShuffleManager + import scala.collection.mutable import scala.language.reflectiveCalls @@ -42,7 +44,9 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // so we coerce consolidation if not already enabled. testConf.set("spark.shuffle.consolidateFiles", "true") - val shuffleBlockManager = new ShuffleBlockManager(null) { + private val shuffleManager = new HashShuffleManager(testConf.clone) + + val shuffleBlockManager = new ShuffleBlockManager(null, shuffleManager) { override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) @@ -148,7 +152,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before actorSystem.actorOf(Props(new BlockManagerMasterActor(true, confCopy, new LiveListenerBus))), confCopy) val store = new BlockManager("", actorSystem, master , serializer, confCopy, - securityManager, null) + securityManager, null, shuffleManager) try { From 8c79574462eed113fc59d4323eedfc55c6e95c06 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 16 Aug 2014 11:26:51 -0700 Subject: [PATCH 007/192] [SQL] Using safe floating-point numbers in doctest Test code in `sql.py` tries to compare two floating-point numbers directly, and cased [build failure(s)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18365/consoleFull). [Doctest documentation](https://docs.python.org/3/library/doctest.html#warnings) recommends using numbers in the form of `I/2**J` to avoid the precision issue. Author: Cheng Lian Closes #1925 from liancheng/fix-pysql-fp-test and squashes the following commits: 0fbf584 [Cheng Lian] Removed unnecessary `...' from inferSchema doctest e8059d4 [Cheng Lian] Using safe floating-point numbers in doctest (cherry picked from commit b4a05928e95c0f6973fd21e60ff9c108f226e38c) Signed-off-by: Michael Armbrust --- python/pyspark/sql.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 95086a2258222..d4ca0cc8f336e 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1093,8 +1093,8 @@ def applySchema(self, rdd, schema): >>> sqlCtx.sql( ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + - ... "float + 1.1 as float FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.1...)] + ... "float + 1.5 as float FROM table2").collect() + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] >>> rdd = sc.parallelize([(127, -32768, 1.0, ... datetime(2010, 1, 1, 1, 1, 1), From bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 16 Aug 2014 14:15:58 -0700 Subject: [PATCH 008/192] [SPARK-2677] BasicBlockFetchIterator#next can wait forever Author: Kousuke Saruta Closes #1632 from sarutak/SPARK-2677 and squashes the following commits: cddbc7b [Kousuke Saruta] Removed Exception throwing when ConnectionManager#handleMessage receives ack for non-referenced message d3bd2a8 [Kousuke Saruta] Modified configuration.md for spark.core.connection.ack.timeout e85f88b [Kousuke Saruta] Removed useless synchronized blocks 7ed48be [Kousuke Saruta] Modified ConnectionManager to use ackTimeoutMonitor ConnectionManager-wide 9b620a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0dd9ad3 [Kousuke Saruta] Modified typo in ConnectionManagerSuite.scala 7cbb8ca [Kousuke Saruta] Modified to match with scalastyle 8a73974 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 ade279a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0174d6a [Kousuke Saruta] Modified ConnectionManager.scala to handle the case remote Executor cannot ack a454239 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 9b7b7c1 [Kousuke Saruta] (WIP) Modifying ConnectionManager.scala (cherry picked from commit 76fa0eaf515fd6771cdd69422b1259485debcae5) Signed-off-by: Josh Rosen --- .../spark/network/ConnectionManager.scala | 45 ++++++++++++++----- .../network/ConnectionManagerSuite.scala | 44 +++++++++++++++++- docs/configuration.md | 9 ++++ 3 files changed, 87 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 95f96b8463a01..37d69a9ec4ce4 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -22,6 +22,7 @@ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ import java.net._ +import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} @@ -61,17 +62,17 @@ private[spark] class ConnectionManager( var ackMessage: Option[Message] = None def markDone(ackMessage: Option[Message]) { - this.synchronized { - this.ackMessage = ackMessage - completionHandler(this) - } + this.ackMessage = ackMessage + completionHandler(this) } } private val selector = SelectorProvider.provider.openSelector() + private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true) // default to 30 second timeout waiting for authentication private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) + private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) private val handleMessageExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.handler.threads.min", 20), @@ -652,19 +653,27 @@ private[spark] class ConnectionManager( } } if (bufferMessage.hasAckId()) { - val sentMessageStatus = messageStatuses.synchronized { + messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { case Some(status) => { messageStatuses -= bufferMessage.ackId - status + status.markDone(Some(message)) } case None => { - throw new Exception("Could not find reference for received ack message " + - message.id) + /** + * We can fall down on this code because of following 2 cases + * + * (1) Invalid ack sent due to buggy code. + * + * (2) Late-arriving ack for a SendMessageStatus + * To avoid unwilling late-arriving ack + * caused by long pause like GC, you can set + * larger value than default to spark.core.connection.ack.wait.timeout + */ + logWarning(s"Could not find reference for received ack Message ${message.id}") } } } - sentMessageStatus.markDone(Some(message)) } else { var ackMessage : Option[Message] = None try { @@ -836,9 +845,23 @@ private[spark] class ConnectionManager( def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) : Future[Message] = { val promise = Promise[Message]() + + val timeoutTask = new TimerTask { + override def run(): Unit = { + messageStatuses.synchronized { + messageStatuses.remove(message.id).foreach ( s => { + promise.failure( + new IOException(s"sendMessageReliably failed because ack " + + "was not received within ${ackTimeout} sec")) + }) + } + } + } + val status = new MessageStatus(message, connectionManagerId, s => { + timeoutTask.cancel() s.ackMessage match { - case None => // Indicates a failure where we either never sent or never got ACK'd + case None => // Indicates a failure where we either never sent or never got ACK'd promise.failure(new IOException("sendMessageReliably failed without being ACK'd")) case Some(ackMessage) => if (ackMessage.hasError) { @@ -852,6 +875,8 @@ private[spark] class ConnectionManager( messageStatuses.synchronized { messageStatuses += ((message.id, status)) } + + ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000) sendMessage(connectionManagerId, message) promise.future } diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala index 846537df003df..e2f4d4c57cdb5 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala @@ -19,14 +19,19 @@ package org.apache.spark.network import java.io.IOException import java.nio._ +import java.util.concurrent.TimeoutException import org.apache.spark.{SecurityManager, SparkConf} import org.scalatest.FunSuite +import org.mockito.Mockito._ +import org.mockito.Matchers._ + +import scala.concurrent.TimeoutException import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ import scala.language.postfixOps -import scala.util.Try +import scala.util.{Failure, Success, Try} /** * Test the ConnectionManager with various security settings. @@ -255,5 +260,42 @@ class ConnectionManagerSuite extends FunSuite { } + test("sendMessageReliably timeout") { + val clientConf = new SparkConf + clientConf.set("spark.authenticate", "false") + val ackTimeout = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + + val clientSecurityManager = new SecurityManager(clientConf) + val manager = new ConnectionManager(0, clientConf, clientSecurityManager) + + val serverConf = new SparkConf + serverConf.set("spark.authenticate", "false") + val serverSecurityManager = new SecurityManager(serverConf) + val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + // sleep 60 sec > ack timeout for simulating server slow down or hang up + Thread.sleep(ackTimeout * 3 * 1000) + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + + val future = manager.sendMessageReliably(managerServer.id, bufferMessage) + + // Future should throw IOException in 30 sec. + // Otherwise TimeoutExcepton is thrown from Await.result. + // We expect TimeoutException is not thrown. + intercept[IOException] { + Await.result(future, (ackTimeout * 2) second) + } + + manager.stop() + managerServer.stop() + } + } diff --git a/docs/configuration.md b/docs/configuration.md index c408c468dcd94..981170d8b49b7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -884,6 +884,15 @@ Apart from these, the following properties are also available, and may be useful out and giving up. + + spark.core.connection.ack.wait.timeout + 60 + + Number of seconds for the connection to wait for ack to occur before timing + out and giving up. To avoid unwilling timeout caused by long pause like GC, + you can set larger value. + + spark.ui.filters None From 0b354be2f9ec35547a60591acf4f4773a4869690 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 15:13:34 -0700 Subject: [PATCH 009/192] [SPARK-3048][MLLIB] add LabeledPoint.parse and remove loadStreamingLabeledPoints Move `parse()` from `LabeledPointParser` to `LabeledPoint` and make it public. This breaks binary compatibility only when a user uses synthesized methods like `tupled` and `curried`, which is rare. `LabeledPoint.parse` is more consistent with `Vectors.parse`, which is why `LabeledPointParser` is not preferred. freeman-lab tdas Author: Xiangrui Meng Closes #1952 from mengxr/labelparser and squashes the following commits: c818fb2 [Xiangrui Meng] merge master ce20e6f [Xiangrui Meng] update mima excludes b386b8d [Xiangrui Meng] fix tests 2436b3d [Xiangrui Meng] add parse() to LabeledPoint (cherry picked from commit 7e70708a99949549adde00cb6246a9582bbc4929) Signed-off-by: Xiangrui Meng --- .../mllib/StreamingLinearRegression.scala | 7 +++---- .../spark/mllib/regression/LabeledPoint.scala | 2 +- .../StreamingLinearRegressionWithSGD.scala | 2 +- .../org/apache/spark/mllib/util/MLUtils.scala | 17 ++--------------- .../mllib/regression/LabeledPointSuite.scala | 4 ++-- .../StreamingLinearRegressionSuite.scala | 6 +++--- project/MimaExcludes.scala | 5 +++++ 7 files changed, 17 insertions(+), 26 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index 1fd37edfa7427..0e992fa9967bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -18,8 +18,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD +import org.apache.spark.mllib.regression.{LabeledPoint, StreamingLinearRegressionWithSGD} import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -56,8 +55,8 @@ object StreamingLinearRegression { val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) - val trainingData = MLUtils.loadStreamingLabeledPoints(ssc, args(0)) - val testData = MLUtils.loadStreamingLabeledPoints(ssc, args(1)) + val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse) + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 62a03af4a9964..17c753c56681f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -36,7 +36,7 @@ case class LabeledPoint(label: Double, features: Vector) { /** * Parser for [[org.apache.spark.mllib.regression.LabeledPoint]]. */ -private[mllib] object LabeledPointParser { +object LabeledPoint { /** * Parses a string resulted from `LabeledPoint#toString` into * an [[org.apache.spark.mllib.regression.LabeledPoint]]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index 8851097050318..1d11fde24712c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vector /** * Train or predict a linear regression model on streaming data. Training uses diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index f4cce86a65ba7..ca35100aa99c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliSampler -import org.apache.spark.mllib.regression.{LabeledPointParser, LabeledPoint} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext @@ -185,7 +185,7 @@ object MLUtils { * @return labeled points stored as an RDD[LabeledPoint] */ def loadLabeledPoints(sc: SparkContext, path: String, minPartitions: Int): RDD[LabeledPoint] = - sc.textFile(path, minPartitions).map(LabeledPointParser.parse) + sc.textFile(path, minPartitions).map(LabeledPoint.parse) /** * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile` with the default number of @@ -194,19 +194,6 @@ object MLUtils { def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = loadLabeledPoints(sc, dir, sc.defaultMinPartitions) - /** - * Loads streaming labeled points from a stream of text files - * where points are in the same format as used in `RDD[LabeledPoint].saveAsTextFile`. - * See `StreamingContext.textFileStream` for more details on how to - * generate a stream from files - * - * @param ssc Streaming context - * @param dir Directory path in any Hadoop-supported file system URI - * @return Labeled points stored as a DStream[LabeledPoint] - */ - def loadStreamingLabeledPoints(ssc: StreamingContext, dir: String): DStream[LabeledPoint] = - ssc.textFileStream(dir).map(LabeledPointParser.parse) - /** * Load labeled data from a file. The data format used here is * , ... diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index d9308aaba6ee1..110c44a7193fd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -28,12 +28,12 @@ class LabeledPointSuite extends FunSuite { LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) points.foreach { p => - assert(p === LabeledPointParser.parse(p.toString)) + assert(p === LabeledPoint.parse(p.toString)) } } test("parse labeled points with v0.9 format") { - val point = LabeledPointParser.parse("1.0,1.0 0.0 -2.0") + val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index ed21f84472c9a..45e25eecf508e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -26,7 +26,7 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext, MLUtils} +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils @@ -55,7 +55,7 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { val numBatches = 10 val batchDuration = Milliseconds(1000) val ssc = new StreamingContext(sc, batchDuration) - val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val data = ssc.textFileStream(testDir.toString).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) .setStepSize(0.1) @@ -97,7 +97,7 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { val batchDuration = Milliseconds(2000) val ssc = new StreamingContext(sc, batchDuration) val numBatches = 5 - val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val data = ssc.textFileStream(testDir.toString()).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) .setStepSize(0.1) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bbe68b29d2d8e..300589394b96f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -129,6 +129,11 @@ object MimaExcludes { Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") ) ++ + Seq( // synthetic methods generated in LabeledPoint + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.regression.LabeledPoint$"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.regression.LabeledPoint.apply"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LabeledPoint.toString") + ) ++ Seq ( // Scala 2.11 compatibility fix ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.$default$2") ) From a12d3ae3223535e6e4c774e4a289b8b2f2e5228b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 15:14:43 -0700 Subject: [PATCH 010/192] [SPARK-3081][MLLIB] rename RandomRDDGenerators to RandomRDDs `RandomRDDGenerators` means factory for `RandomRDDGenerator`. However, its methods return RDDs but not RDDGenerators. So a more proper (and shorter) name would be `RandomRDDs`. dorx brkyvz Author: Xiangrui Meng Closes #1979 from mengxr/randomrdds and squashes the following commits: b161a2d [Xiangrui Meng] rename RandomRDDGenerators to RandomRDDs (cherry picked from commit ac6411c6e75906997c78de23dfdbc8d225b87cfd) Signed-off-by: Xiangrui Meng --- .../mllib/api/python/PythonMLLibAPI.scala | 2 +- ...omRDDGenerators.scala => RandomRDDs.scala} | 6 ++--- ...atorsSuite.scala => RandomRDDsSuite.scala} | 16 ++++++------ python/pyspark/mllib/random.py | 25 +++++++++---------- 4 files changed, 24 insertions(+), 25 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/random/{RandomRDDGenerators.scala => RandomRDDs.scala} (99%) rename mllib/src/test/scala/org/apache/spark/mllib/random/{RandomRDDGeneratorsSuite.scala => RandomRDDsSuite.scala} (88%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 18dc087856785..4343124f102a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} +import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala rename to mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index b0a0593223910..36270369526cd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.random +import scala.reflect.ClassTag + import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector @@ -24,14 +26,12 @@ import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -import scala.reflect.ClassTag - /** * :: Experimental :: * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. */ @Experimental -object RandomRDDGenerators { +object RandomRDDs { /** * :: Experimental :: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala similarity index 88% rename from mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 96e0bc63b0fa4..c50b78bcbcc61 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.StatCounter * * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged */ -class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Serializable { +class RandomRDDsSuite extends FunSuite with LocalSparkContext with Serializable { def testGeneratedRDD(rdd: RDD[Double], expectedSize: Long, @@ -113,18 +113,18 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri val poissonMean = 100.0 for (seed <- 0 until 5) { - val uniform = RandomRDDGenerators.uniformRDD(sc, size, numPartitions, seed) + val uniform = RandomRDDs.uniformRDD(sc, size, numPartitions, seed) testGeneratedRDD(uniform, size, numPartitions, 0.5, 1 / math.sqrt(12)) - val normal = RandomRDDGenerators.normalRDD(sc, size, numPartitions, seed) + val normal = RandomRDDs.normalRDD(sc, size, numPartitions, seed) testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) - val poisson = RandomRDDGenerators.poissonRDD(sc, poissonMean, size, numPartitions, seed) + val poisson = RandomRDDs.poissonRDD(sc, poissonMean, size, numPartitions, seed) testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) } // mock distribution to check that partitions have unique seeds - val random = RandomRDDGenerators.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) + val random = RandomRDDs.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) assert(random.collect.size === random.collect.distinct.size) } @@ -135,13 +135,13 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri val poissonMean = 100.0 for (seed <- 0 until 5) { - val uniform = RandomRDDGenerators.uniformVectorRDD(sc, rows, cols, parts, seed) + val uniform = RandomRDDs.uniformVectorRDD(sc, rows, cols, parts, seed) testGeneratedVectorRDD(uniform, rows, cols, parts, 0.5, 1 / math.sqrt(12)) - val normal = RandomRDDGenerators.normalVectorRDD(sc, rows, cols, parts, seed) + val normal = RandomRDDs.normalVectorRDD(sc, rows, cols, parts, seed) testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) - val poisson = RandomRDDGenerators.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) + val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) } } diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index eb496688b6eef..3f3b19053d32e 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -25,8 +25,7 @@ from pyspark.serializers import NoOpSerializer -class RandomRDDGenerators: - +class RandomRDDs: """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. @@ -40,17 +39,17 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - C{RandomRDDGenerators.uniformRDD(sc, n, p, seed)\ + C{RandomRDDs.uniformRDD(sc, n, p, seed)\ .map(lambda v: a + (b - a) * v)} - >>> x = RandomRDDGenerators.uniformRDD(sc, 100).collect() + >>> x = RandomRDDs.uniformRDD(sc, 100).collect() >>> len(x) 100 >>> max(x) <= 1.0 and min(x) >= 0.0 True - >>> RandomRDDGenerators.uniformRDD(sc, 100, 4).getNumPartitions() + >>> RandomRDDs.uniformRDD(sc, 100, 4).getNumPartitions() 4 - >>> parts = RandomRDDGenerators.uniformRDD(sc, 100, seed=4).getNumPartitions() + >>> parts = RandomRDDs.uniformRDD(sc, 100, seed=4).getNumPartitions() >>> parts == sc.defaultParallelism True """ @@ -66,10 +65,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): To transform the distribution in the generated RDD from standard normal to some other normal N(mean, sigma), use - C{RandomRDDGenerators.normal(sc, n, p, seed)\ + C{RandomRDDs.normal(sc, n, p, seed)\ .map(lambda v: mean + sigma * v)} - >>> x = RandomRDDGenerators.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) >>> stats = x.stats() >>> stats.count() 1000L @@ -89,7 +88,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): distribution with the input mean. >>> mean = 100.0 - >>> x = RandomRDDGenerators.poissonRDD(sc, mean, 1000, seed=1L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=1L) >>> stats = x.stats() >>> stats.count() 1000L @@ -110,12 +109,12 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): from the uniform distribution on [0.0 1.0]. >>> import numpy as np - >>> mat = np.matrix(RandomRDDGenerators.uniformVectorRDD(sc, 10, 10).collect()) + >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) >>> mat.shape (10, 10) >>> mat.max() <= 1.0 and mat.min() >= 0.0 True - >>> RandomRDDGenerators.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() + >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ jrdd = sc._jvm.PythonMLLibAPI() \ @@ -130,7 +129,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): from the standard normal distribution. >>> import numpy as np - >>> mat = np.matrix(RandomRDDGenerators.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -151,7 +150,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDGenerators.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) From 721f2fdc95032132af3d4a00dbc8399d356f8faf Mon Sep 17 00:00:00 2001 From: iAmGhost Date: Sat, 16 Aug 2014 16:48:38 -0700 Subject: [PATCH 011/192] [SPARK-3035] Wrong example with SparkContext.addFile https://issues.apache.org/jira/browse/SPARK-3035 fix for wrong document. Author: iAmGhost Closes #1942 from iAmGhost/master and squashes the following commits: 487528a [iAmGhost] [SPARK-3035] Wrong example with SparkContext.addFile fix for wrong document. (cherry picked from commit 379e7585c356f20bf8b4878ecba9401e2195da12) Signed-off-by: Josh Rosen --- python/pyspark/context.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4001ecab5ea00..6c049238819a7 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -613,7 +613,7 @@ def addFile(self, path): >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: ... fileVal = int(testFile.readline()) - ... return [x * 100 for x in iterator] + ... return [x * fileVal for x in iterator] >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect() [100, 200, 300, 400] """ From 5dd571c29ef97cadd23a54fcf4d5de869e3c56bc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 16 Aug 2014 16:59:34 -0700 Subject: [PATCH 012/192] [SPARK-1065] [PySpark] improve supporting for large broadcast Passing large object by py4j is very slow (cost much memory), so pass broadcast objects via files (similar to parallelize()). Add an option to keep object in driver (it's False by default) to save memory in driver. Author: Davies Liu Closes #1912 from davies/broadcast and squashes the following commits: e06df4a [Davies Liu] load broadcast from disk in driver automatically db3f232 [Davies Liu] fix serialization of accumulator 631a827 [Davies Liu] Merge branch 'master' into broadcast c7baa8c [Davies Liu] compress serrialized broadcast and command 9a7161f [Davies Liu] fix doc tests e93cf4b [Davies Liu] address comments: add test 6226189 [Davies Liu] improve large broadcast (cherry picked from commit 2fc8aca086a2679b854038b7e2c488f19039ecbd) Signed-off-by: Josh Rosen --- .../apache/spark/api/python/PythonRDD.scala | 8 ++++ python/pyspark/broadcast.py | 37 ++++++++++++++----- python/pyspark/context.py | 20 ++++++---- python/pyspark/rdd.py | 5 ++- python/pyspark/serializers.py | 17 +++++++++ python/pyspark/tests.py | 7 ++++ python/pyspark/worker.py | 8 ++-- 7 files changed, 81 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 9f5c5bd30f0c9..10210a2927dcc 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -315,6 +315,14 @@ private[spark] object PythonRDD extends Logging { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } + def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = { + val file = new DataInputStream(new FileInputStream(filename)) + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + sc.broadcast(obj) + } + def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { // The right way to implement this would be to use TypeTags to get the full // type of T. Since I don't want to introduce breaking changes throughout the diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index f3e64989ed564..675a2fcd2ff4e 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -21,18 +21,16 @@ >>> b = sc.broadcast([1, 2, 3, 4, 5]) >>> b.value [1, 2, 3, 4, 5] - ->>> from pyspark.broadcast import _broadcastRegistry ->>> _broadcastRegistry[b.bid] = b ->>> from cPickle import dumps, loads ->>> loads(dumps(b)).value -[1, 2, 3, 4, 5] - >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] +>>> b.unpersist() >>> large_broadcast = sc.broadcast(list(range(10000))) """ +import os + +from pyspark.serializers import CompressedSerializer, PickleSerializer + # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} @@ -52,17 +50,38 @@ class Broadcast(object): Access its value through C{.value}. """ - def __init__(self, bid, value, java_broadcast=None, pickle_registry=None): + def __init__(self, bid, value, java_broadcast=None, + pickle_registry=None, path=None): """ Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ - self.value = value self.bid = bid + if path is None: + self.value = value self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry + self.path = path + + def unpersist(self, blocking=False): + self._jbroadcast.unpersist(blocking) + os.unlink(self.path) def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) + + def __getattr__(self, item): + if item == 'value' and self.path is not None: + ser = CompressedSerializer(PickleSerializer()) + value = ser.load_stream(open(self.path)).next() + self.value = value + return value + + raise AttributeError(item) + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6c049238819a7..a90870ed3a353 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer + PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -566,13 +566,19 @@ def broadcast(self, value): """ Broadcast a read-only variable to the cluster, returning a L{Broadcast} - object for reading it in distributed functions. The variable will be - sent to each cluster only once. + object for reading it in distributed functions. The variable will + be sent to each cluster only once. + + :keep: Keep the `value` in driver or not. """ - pickleSer = PickleSerializer() - pickled = pickleSer.dumps(value) - jbroadcast = self._jsc.broadcast(bytearray(pickled)) - return Broadcast(jbroadcast.id(), value, jbroadcast, self._pickled_broadcast_vars) + ser = CompressedSerializer(PickleSerializer()) + # pass large object by py4j is very slow and need much memory + tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) + ser.dump_stream([value], tempFile) + tempFile.close() + jbroadcast = self._jvm.PythonRDD.readBroadcastFromFile(self._jsc, tempFile.name) + return Broadcast(jbroadcast.id(), None, jbroadcast, + self._pickled_broadcast_vars, tempFile.name) def accumulator(self, value, accum_param=None): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3934bdda0a466..240381e5bae12 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -36,7 +36,7 @@ from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long + PickleSerializer, pack_long, CompressedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -1810,7 +1810,8 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CompressedSerializer(CloudPickleSerializer()) + pickled_command = ser.dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index df90cafb245bf..74870c0edcf99 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -67,6 +67,7 @@ import sys import types import collections +import zlib from pyspark import cloudpickle @@ -403,6 +404,22 @@ def loads(self, obj): raise ValueError("invalid sevialization type: %s" % _type) +class CompressedSerializer(FramedSerializer): + """ + compress the serialized data + """ + + def __init__(self, serializer): + FramedSerializer.__init__(self) + self.serializer = serializer + + def dumps(self, obj): + return zlib.compress(self.serializer.dumps(obj), 1) + + def loads(self, obj): + return self.serializer.loads(zlib.decompress(obj)) + + class UTF8Deserializer(Serializer): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 22b51110ed671..f1fece998cd54 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -323,6 +323,13 @@ def test_namedtuple_in_rdd(self): theDoes = self.sc.parallelize([jon, jane]) self.assertEquals([jon, jane], theDoes.collect()) + def test_large_broadcast(self): + N = 100000 + data = [[float(i) for i in range(300)] for i in range(N)] + bdata = self.sc.broadcast(data) # 270MB + m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() + self.assertEquals(N, m) + class TestIO(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 2770f63059853..77a9c4a0e0677 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,7 +30,8 @@ from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ - write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer + write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ + CompressedSerializer pickleSer = PickleSerializer() @@ -65,12 +66,13 @@ def main(infile, outfile): # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) + ser = CompressedSerializer(pickleSer) for _ in range(num_broadcast_variables): bid = read_long(infile) - value = pickleSer._read_with_length(infile) + value = ser._read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, value) - command = pickleSer._read_with_length(infile) + command = ser._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) From f02e327f0bc975e7f33092e449bc0edd95f95580 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 16 Aug 2014 20:05:55 -0700 Subject: [PATCH 013/192] In the stop method of ConnectionManager to cancel the ackTimeoutMonitor cc JoshRosen sarutak Author: GuoQiang Li Closes #1989 from witgo/cancel_ackTimeoutMonitor and squashes the following commits: 4a700fa [GuoQiang Li] In the stop method of ConnectionManager to cancel the ackTimeoutMonitor (cherry picked from commit bc95fe08dff62a0abea314ab4ab9275c8f119598) Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 37d69a9ec4ce4..e77d762bdf221 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -886,6 +886,7 @@ private[spark] class ConnectionManager( } def stop() { + ackTimeoutMonitor.cancel() selectorThread.interrupt() selectorThread.join() selector.close() From 413a329e186de2ec96f80f614c36678bee6f332f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 21:16:27 -0700 Subject: [PATCH 014/192] [SPARK-3077][MLLIB] fix some chisq-test - promote nullHypothesis field in ChiSqTestResult to TestResult. Every test should have a null hypothesis - correct null hypothesis statement for independence test - p-value: 0.01 -> 0.1 Author: Xiangrui Meng Closes #1982 from mengxr/fix-chisq and squashes the following commits: 5f0de02 [Xiangrui Meng] make ChiSqTestResult constructor package private bc74ea1 [Xiangrui Meng] update chisq-test (cherry picked from commit fbad72288d8b6e641b00417a544cae6e8bfef2d7) Signed-off-by: Xiangrui Meng --- .../spark/mllib/stat/test/ChiSqTest.scala | 2 +- .../spark/mllib/stat/test/TestResult.scala | 28 +++++++++++-------- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 8f6752737402e..215de95db5113 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -56,7 +56,7 @@ private[stat] object ChiSqTest extends Logging { object NullHypothesis extends Enumeration { type NullHypothesis = Value val goodnessOfFit = Value("observed follows the same distribution as expected.") - val independence = Value("observations in each column are statistically independent.") + val independence = Value("the occurrence of the outcomes is statistically independent.") } // Method identification based on input methodName string diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala index 2f278621335e1..4784f9e947908 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -44,6 +44,11 @@ trait TestResult[DF] { */ def statistic: Double + /** + * Null hypothesis of the test. + */ + def nullHypothesis: String + /** * String explaining the hypothesis test result. * Specific classes implementing this trait should override this method to output test-specific @@ -53,13 +58,13 @@ trait TestResult[DF] { // String explaining what the p-value indicates. val pValueExplain = if (pValue <= 0.01) { - "Very strong presumption against null hypothesis." + s"Very strong presumption against null hypothesis: $nullHypothesis." } else if (0.01 < pValue && pValue <= 0.05) { - "Strong presumption against null hypothesis." - } else if (0.05 < pValue && pValue <= 0.01) { - "Low presumption against null hypothesis." + s"Strong presumption against null hypothesis: $nullHypothesis." + } else if (0.05 < pValue && pValue <= 0.1) { + s"Low presumption against null hypothesis: $nullHypothesis." } else { - "No presumption against null hypothesis." + s"No presumption against null hypothesis: $nullHypothesis." } s"degrees of freedom = ${degreesOfFreedom.toString} \n" + @@ -70,19 +75,18 @@ trait TestResult[DF] { /** * :: Experimental :: - * Object containing the test results for the chi squared hypothesis test. + * Object containing the test results for the chi-squared hypothesis test. */ @Experimental -class ChiSqTestResult(override val pValue: Double, +class ChiSqTestResult private[stat] (override val pValue: Double, override val degreesOfFreedom: Int, override val statistic: Double, val method: String, - val nullHypothesis: String) extends TestResult[Int] { + override val nullHypothesis: String) extends TestResult[Int] { override def toString: String = { - "Chi squared test summary: \n" + - s"method: $method \n" + - s"null hypothesis: $nullHypothesis \n" + - super.toString + "Chi squared test summary:\n" + + s"method: $method\n" + + super.toString } } From 91af120b4391656cb8f7b2300202dc622c032c33 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sat, 16 Aug 2014 23:53:14 -0700 Subject: [PATCH 015/192] [SPARK-3042] [mllib] DecisionTree Filter top-down instead of bottom-up DecisionTree needs to match each example to a node at each iteration. It currently does this with a set of filters very inefficiently: For each example, it examines each node at the current level and traces up to the root to see if that example should be handled by that node. Fix: Filter top-down using the partly built tree itself. Major changes: * Eliminated Filter class, findBinsForLevel() method. * Set up node parent links in main loop over levels in train(). * Added predictNodeIndex() for filtering top-down. * Added DTMetadata class Other changes: * Pre-compute set of unorderedFeatures. Notes for following expected PR based on [https://issues.apache.org/jira/browse/SPARK-3043]: * The unorderedFeatures set will next be stored in a metadata structure to simplify function calls (to store other items such as the data in strategy). I've done initial tests indicating that this speeds things up, but am only now running large-scale ones. CC: mengxr manishamde chouqin Any comments are welcome---thanks! Author: Joseph K. Bradley Closes #1975 from jkbradley/dt-opt2 and squashes the following commits: a0ed0da [Joseph K. Bradley] Renamed DTMetadata to DecisionTreeMetadata. Small doc updates. 3726d20 [Joseph K. Bradley] Small code improvements based on code review. ac0b9f8 [Joseph K. Bradley] Small updates based on code review. Main change: Now using << instead of math.pow. db0d773 [Joseph K. Bradley] scala style fix 6a38f48 [Joseph K. Bradley] Added DTMetadata class for cleaner code 931a3a7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 797f68a [Joseph K. Bradley] Fixed DecisionTreeSuite bug for training second level. Needed to update treePointToNodeIndex with groupShift. f40381c [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 5f2dec2 [Joseph K. Bradley] Fixed scalastyle issue in TreePoint 6b5651e [Joseph K. Bradley] Updates based on code review. 1 major change: persisting to memory + disk, not just memory. 2d2aaaf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals (cherry picked from commit 73ab7f141c205df277c6ac19252e590d6806c41f) Signed-off-by: Xiangrui Meng --- .../spark/mllib/tree/DecisionTree.scala | 878 ++++++++---------- .../tree/impl/DecisionTreeMetadata.scala | 101 ++ .../spark/mllib/tree/impl/TreePoint.scala | 30 +- .../apache/spark/mllib/tree/model/Bin.scala | 18 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../spark/mllib/tree/model/Filter.scala | 28 - .../apache/spark/mllib/tree/model/Node.scala | 16 +- .../apache/spark/mllib/tree/model/Split.scala | 5 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 167 ++-- 9 files changed, 615 insertions(+), 630 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala 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 2a3107a13e916..6b9a8f72c244e 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 @@ -27,7 +27,7 @@ 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.impl.{TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD @@ -62,43 +62,38 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo timer.start("init") val retaggedInput = input.retag(classOf[LabeledPoint]) + val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. timer.start("findSplitsBins") - val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) val numBins = bins(0).length timer.stop("findSplitsBins") logDebug("numBins = " + numBins) + // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. - val treeInput = TreePoint.convertToTreeRDD(retaggedInput, strategy, bins) + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) .persist(StorageLevel.MEMORY_AND_DISK) + val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree - val maxNumNodes = math.pow(2, maxDepth + 1).toInt - 1 - // Initialize an array to hold filters applied to points for each node. - val filters = new Array[List[Filter]](maxNumNodes) - // The filter at the top node is an empty list. - filters(0) = List() + val maxNumNodes = (2 << maxDepth) - 1 // Initialize an array to hold parent impurity calculations for each node. val parentImpurities = new Array[Double](maxNumNodes) // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) - // num features - val numFeatures = treeInput.take(1)(0).binnedFeatures.size // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = DecisionTree.getElementsPerNode(numFeatures, numBins, - strategy.numClassesForClassification, strategy.isMulticlassWithCategoricalFeatures, - strategy.algo) + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -114,9 +109,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo /* * The main idea here is to perform level-wise training of the decision tree nodes thus * reducing the passes over the data from l to log2(l) where l is the total number of nodes. - * Each data sample is checked for validity w.r.t to each node at a given level -- i.e., - * the sample is only used for the split calculation at the node if the sampled would have - * still survived the filters of the parent nodes. + * Each data sample is handled by a particular node at that level (or it reaches a leaf + * beforehand and is not used in later levels. */ var level = 0 @@ -130,22 +124,37 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, - strategy, level, filters, splits, bins, maxLevelForSingleGroup, timer) + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") + val levelNodeIndexOffset = (1 << level) - 1 for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + val nodeIndex = levelNodeIndexOffset + index + val isLeftChild = level != 0 && nodeIndex % 2 == 1 + val parentNodeIndex = if (isLeftChild) { // -1 for root node + (nodeIndex - 1) / 2 + } else { + (nodeIndex - 2) / 2 + } + // Extract info for this node (index) at the current level. timer.start("extractNodeInfo") - // Extract info for nodes at the current level. extractNodeInfo(nodeSplitStats, level, index, nodes) timer.stop("extractNodeInfo") - timer.start("extractInfoForLowerLevels") + if (level != 0) { + // Set parent. + if (isLeftChild) { + nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) + } else { + nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) + } + } // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) + timer.start("extractInfoForLowerLevels") + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) timer.stop("extractInfoForLowerLevels") logDebug("final best split = " + nodeSplitStats._1) } - require(math.pow(2, level) == splitsStatsForLevel.length) + require((1 << level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -183,7 +192,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo nodes: Array[Node]): Unit = { val split = nodeSplitStats._1 val stats = nodeSplitStats._2 - val nodeIndex = math.pow(2, level).toInt - 1 + index + val nodeIndex = (1 << level) - 1 + index val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) @@ -198,31 +207,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo index: Int, maxDepth: Int, nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double], - filters: Array[List[Filter]]): Unit = { - // 0 corresponds to the left child node and 1 corresponds to the right child node. - var i = 0 - while (i <= 1) { - // Calculate the index of the node from the node level and the index at the current level. - val nodeIndex = math.pow(2, level + 1).toInt - 1 + 2 * index + i - if (level < maxDepth) { - val impurity = if (i == 0) { - nodeSplitStats._2.leftImpurity - } else { - nodeSplitStats._2.rightImpurity - } - logDebug("nodeIndex = " + nodeIndex + ", impurity = " + impurity) - // noting the parent impurities - parentImpurities(nodeIndex) = impurity - // noting the parents filters for the child nodes - val childFilter = new Filter(nodeSplitStats._1, if (i == 0) -1 else 1) - filters(nodeIndex) = childFilter :: filters((nodeIndex - 1) / 2) - for (filter <- filters(nodeIndex)) { - logDebug("Filter = " + filter) - } - } - i += 1 + parentImpurities: Array[Double]): Unit = { + + if (level >= maxDepth) { + return } + + val leftNodeIndex = (2 << level) - 1 + 2 * index + val leftImpurity = nodeSplitStats._2.leftImpurity + logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) + parentImpurities(leftNodeIndex) = leftImpurity + + val rightNodeIndex = leftNodeIndex + 1 + val rightImpurity = nodeSplitStats._2.rightImpurity + logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) + parentImpurities(rightNodeIndex) = rightImpurity } } @@ -434,10 +433,8 @@ object DecisionTree extends Serializable with Logging { * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for constructing the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features * @param bins possible bins for all features * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. @@ -446,9 +443,9 @@ object DecisionTree extends Serializable with Logging { protected[tree] def findBestSplits( input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, @@ -459,34 +456,32 @@ object DecisionTree extends Serializable with Logging { // the nodes are divided into multiple groups at each level with the number of groups // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = math.pow(2, level - maxLevelForSingleGroup).toInt + val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) var bestSplits = new Array[(Split, InformationGainStats)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { - val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, strategy, level, - filters, splits, bins, timer, numGroups, groupIndex) + val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, metadata, level, + nodes, splits, bins, timer, numGroups, groupIndex) bestSplits = Array.concat(bestSplits, bestSplitsForGroup) groupIndex += 1 } bestSplits } else { - findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins, timer) + findBestSplitsPerGroup(input, parentImpurities, metadata, level, nodes, splits, bins, timer) } } - /** + /** * Returns an array of optimal splits for a group of nodes at a given level * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for constructing the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features - * @param bins possible bins for all features + * @param bins possible bins for all features, indexed as (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. @@ -494,9 +489,9 @@ object DecisionTree extends Serializable with Logging { private def findBestSplitsPerGroup( input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], timer: TimeTracker, @@ -515,7 +510,7 @@ object DecisionTree extends Serializable with Logging { * We use a bin-wise best split computation strategy instead of a straightforward best split * computation strategy. Instead of analyzing each sample for contribution to the left/right * child node impurity of every split, we first categorize each feature of a sample into a - * bin. Each bin is an interval between a low and high split. Since each splits, and thus bin, + * bin. Each bin is an interval between a low and high split. Since each split, and thus bin, * is ordered (read ordering for categorical variables in the findSplitsBins method), * we exploit this structure to calculate aggregates for bins and then use these aggregates * to calculate information gain for each split. @@ -531,160 +526,124 @@ object DecisionTree extends Serializable with Logging { // numNodes: Number of nodes in this (level of tree, group), // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = math.pow(2, level).toInt / numGroups + val numNodes = (1 << level) / numGroups logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. - val numFeatures = input.first().binnedFeatures.size + val numFeatures = metadata.numFeatures logDebug("numFeatures = " + numFeatures) // numBins: Number of bins = 1 + number of possible splits val numBins = bins(0).length logDebug("numBins = " + numBins) - val numClasses = strategy.numClassesForClassification + val numClasses = metadata.numClasses logDebug("numClasses = " + numClasses) - val isMulticlassClassification = strategy.isMulticlassClassification - logDebug("isMulticlassClassification = " + isMulticlassClassification) + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) - val isMulticlassClassificationWithCategoricalFeatures - = strategy.isMulticlassWithCategoricalFeatures - logDebug("isMultiClassWithCategoricalFeatures = " + - isMulticlassClassificationWithCategoricalFeatures) + val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures + logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** Find the filters used before reaching the current code. */ - def findParentFilters(nodeIndex: Int): List[Filter] = { - if (level == 0) { - List[Filter]() - } else { - val nodeFilterIndex = math.pow(2, level).toInt - 1 + nodeIndex + groupShift - filters(nodeFilterIndex) - } - } - /** - * Find whether the sample is valid input for the current node, i.e., whether it passes through - * all the filters for the current node. + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. */ - def isSampleValid(parentFilters: List[Filter], treePoint: TreePoint): Boolean = { - // leaf - if ((level > 0) && (parentFilters.length == 0)) { - return false - } - - // Apply each filter and check sample validity. Return false when invalid condition found. - parentFilters.foreach { filter => - val featureIndex = filter.split.feature - val comparison = filter.comparison - val isFeatureContinuous = filter.split.featureType == Continuous - if (isFeatureContinuous) { - val binId = treePoint.binnedFeatures(featureIndex) - val bin = bins(featureIndex)(binId) - val featureValue = bin.highSplit.threshold - val threshold = filter.split.threshold - comparison match { - case -1 => if (featureValue > threshold) return false - case 1 => if (featureValue <= threshold) return false + def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold } - } else { - val numFeatureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits = - numBins > math.pow(2, numFeatureCategories.toInt - 1) - 1 - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - val featureValue = if (isUnorderedFeature) { - treePoint.binnedFeatures(featureIndex) + case Categorical => { + val featureValue = if (metadata.isUnordered(featureIndex)) { + binnedFeatures(featureIndex) + } else { + val binIndex = binnedFeatures(featureIndex) + bins(featureIndex)(binIndex).category + } + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + node.id * 2 + 1 // left } else { - val binId = treePoint.binnedFeatures(featureIndex) - bins(featureIndex)(binId).category + node.id * 2 + 2 // right } - val containsFeature = filter.split.categories.contains(featureValue) - comparison match { - case -1 => if (!containsFeature) return false - case 1 => if (containsFeature) return false + } else { + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures) } } } + } - // Return true when the sample is valid for all filters. - true + def nodeIndexToLevel(idx: Int): Int = { + if (idx == 0) { + 0 + } else { + math.floor(math.log(idx) / math.log(2)).toInt + } } + // Used for treePointToNodeIndex + val levelOffset = (1 << level) - 1 + /** - * Finds bins for all nodes (and all features) at a given level. - * For l nodes, k features the storage is as follows: - * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk, - * where b_ij is an integer between 0 and numBins - 1 for regressions and binary - * classification and the categorical feature value in multiclass classification. - * Invalid sample is denoted by noting bin for feature 1 as -1. - * - * For unordered features, the "bin index" returned is actually the feature value (category). - * - * @return Array of size 1 + numFeatures * numNodes, where - * arr(0) = label for labeledPoint, and - * arr(1 + numFeatures * nodeIndex + featureIndex) = - * bin index for this labeledPoint - * (or InvalidBinIndex if labeledPoint is not handled by this node) + * Find the node index for the given example. + * Nodes are indexed from 0 at the start of this (level, group). + * If the example does not reach this level, returns a value < 0. */ - def findBinsForLevel(treePoint: TreePoint): Array[Double] = { - // Calculate bin index and label per feature per node. - val arr = new Array[Double](1 + (numFeatures * numNodes)) - // First element of the array is the label of the instance. - arr(0) = treePoint.label - // Iterate over nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - val parentFilters = findParentFilters(nodeIndex) - // Find out whether the sample qualifies for the particular node. - val sampleValid = isSampleValid(parentFilters, treePoint) - val shift = 1 + numFeatures * nodeIndex - if (!sampleValid) { - // Mark one bin as -1 is sufficient. - arr(shift) = InvalidBinIndex - } else { - var featureIndex = 0 - while (featureIndex < numFeatures) { - arr(shift + featureIndex) = treePoint.binnedFeatures(featureIndex) - featureIndex += 1 - } - } - nodeIndex += 1 + def treePointToNodeIndex(treePoint: TreePoint): Int = { + if (level == 0) { + 0 + } else { + val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) + // Get index for this (level, group). + globalNodeIndex - levelOffset - groupShift } - arr } - // Find feature bins for all nodes at a level. - timer.start("aggregation") - val binMappedRDD = input.map(x => findBinsForLevel(x)) - /** * Increment aggregate in location for (node, feature, bin, label). * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. * @param agg Array storing aggregate calculation, of size: * numClasses * numBins * numFeatures * numNodes. * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ def updateBinForOrderedFeature( - arr: Array[Double], + treePoint: TreePoint, agg: Array[Double], nodeIndex: Int, - label: Double, featureIndex: Int): Unit = { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex // Update the left or right count for one bin. val aggIndex = numClasses * numBins * numFeatures * nodeIndex + numClasses * numBins * featureIndex + - numClasses * arr(arrIndex).toInt + - label.toInt + numClasses * treePoint.binnedFeatures(featureIndex) + + treePoint.label.toInt agg(aggIndex) += 1 } @@ -693,8 +652,8 @@ object DecisionTree extends Serializable with Logging { * where [bins] ranges over all bins. * Updates left or right side of aggregate depending on split. * - * @param arr arr(0) = label. - * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category) + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param treePoint Data point being aggregated. * @param agg Indexed by (left/right, node, feature, bin, label) * where label is the least significant bit. * The left/right specifier is a 0/1 index indicating left/right child info. @@ -703,21 +662,18 @@ object DecisionTree extends Serializable with Logging { def updateBinForUnorderedFeature( nodeIndex: Int, featureIndex: Int, - arr: Array[Double], - label: Double, + treePoint: TreePoint, agg: Array[Double], rightChildShift: Int): Unit = { - // Find the bin index for this feature. - val arrIndex = 1 + numFeatures * nodeIndex + featureIndex - val featureValue = arr(arrIndex).toInt + val featureValue = treePoint.binnedFeatures(featureIndex) // Update the left or right count for one bin. val aggShift = numClasses * numBins * numFeatures * nodeIndex + numClasses * numBins * featureIndex + - label.toInt + treePoint.label.toInt // Find all matching bins and increment their values - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 + val featureCategories = metadata.featureArity(featureIndex) + val numCategoricalBins = (1 << featureCategories - 1) - 1 var binIndex = 0 while (binIndex < numCategoricalBins) { val aggIndex = aggShift + binIndex * numClasses @@ -733,30 +689,21 @@ object DecisionTree extends Serializable with Logging { /** * Helper for binSeqOp. * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). * @param agg Array storing aggregate calculation, of size: * numClasses * numBins * numFeatures * numNodes. * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def binaryOrNotCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - featureIndex += 1 - } - } - nodeIndex += 1 + def binaryOrNotCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) + featureIndex += 1 } } @@ -765,49 +712,28 @@ object DecisionTree extends Serializable with Logging { /** * Helper for binSeqOp. * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). - * For ordered features, - * arr(1 + featureIndex + nodeIndex * numFeatures) = bin index. - * For unordered features, - * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category). * @param agg Array storing aggregate calculation. * For ordered features, this is of size: * numClasses * numBins * numFeatures * numNodes. * For unordered features, this is of size: * 2 * numClasses * numBins * numFeatures * numNodes. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def multiclassWithCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - } else { - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isSpaceSufficientForAllCategoricalSplits) { - updateBinForUnorderedFeature(nodeIndex, featureIndex, arr, label, agg, - rightChildShift) - } else { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - } - } - featureIndex += 1 - } + def multiclassWithCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) + } else { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) } - nodeIndex += 1 + featureIndex += 1 } } @@ -818,36 +744,25 @@ object DecisionTree extends Serializable with Logging { * * @param agg Array storing aggregate calculation, updated by this function. * Size: 3 * numBins * numFeatures * numNodes - * @param arr Bin mapping from findBinsForLevel. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). * @return agg */ - def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex - // Update count, sum, and sum^2 for one bin. - val aggShift = 3 * numBins * numFeatures * nodeIndex - val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3 - agg(aggIndex) = agg(aggIndex) + 1 - agg(aggIndex + 1) = agg(aggIndex + 1) + label - agg(aggIndex + 2) = agg(aggIndex + 2) + label * label - featureIndex += 1 - } - } - nodeIndex += 1 + def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Update count, sum, and sum^2 for one bin. + val binIndex = treePoint.binnedFeatures(featureIndex) + val aggIndex = + 3 * numBins * numFeatures * nodeIndex + + 3 * numBins * featureIndex + + 3 * binIndex + agg(aggIndex) += 1 + agg(aggIndex + 1) += label + agg(aggIndex + 2) += label * label + featureIndex += 1 } } @@ -866,26 +781,30 @@ object DecisionTree extends Serializable with Logging { * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. * Size for regression: * 3 * numBins * numFeatures * numNodes. - * @param arr Bin mapping from findBinsForLevel. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. * @return agg */ - def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { - strategy.algo match { - case Classification => - if(isMulticlassClassificationWithCategoricalFeatures) { - multiclassWithCategoricalBinSeqOp(arr, agg) + def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + val nodeIndex = treePointToNodeIndex(treePoint) + // If the example does not reach this level, then nodeIndex < 0. + // If the example reaches this level but is handled in a different group, + // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). + if (nodeIndex >= 0 && nodeIndex < numNodes) { + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) } else { - binaryOrNotCategoricalBinSeqOp(arr, agg) + binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) } - case Regression => regressionBinSeqOp(arr, agg) + } else { + regressionBinSeqOp(agg, treePoint, nodeIndex) + } } agg } // Calculate bin aggregate length for classification or regression. - val binAggregateLength = numNodes * getElementsPerNode(numFeatures, numBins, numClasses, - isMulticlassClassificationWithCategoricalFeatures, strategy.algo) + val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) logDebug("binAggregateLength = " + binAggregateLength) /** @@ -905,144 +824,134 @@ object DecisionTree extends Serializable with Logging { } // Calculate bin aggregates. + timer.start("aggregation") val binAggregates = { - binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) + input.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) } timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) /** - * Calculates the information gain for all splits based upon left/right split aggregates. - * @param leftNodeAgg left node aggregates - * @param featureIndex feature index - * @param splitIndex split index - * @param rightNodeAgg right node aggregate + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftNodeAgg left node aggregates for this (feature, split) + * @param rightNodeAgg right node aggregate for this (feature, split) * @param topImpurity impurity of the parent node * @return information gain and statistics for all splits */ def calculateGainForSplit( - leftNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int, - splitIndex: Int, - rightNodeAgg: Array[Array[Array[Double]]], + leftNodeAgg: Array[Double], + rightNodeAgg: Array[Double], topImpurity: Double): InformationGainStats = { - strategy.algo match { - case Classification => - val leftCounts: Array[Double] = leftNodeAgg(featureIndex)(splitIndex) - val rightCounts: Array[Double] = rightNodeAgg(featureIndex)(splitIndex) - val leftTotalCount = leftCounts.sum - val rightTotalCount = rightCounts.sum - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val rootNodeCounts = new Array[Double](numClasses) - var classIndex = 0 - while (classIndex < numClasses) { - rootNodeCounts(classIndex) = leftCounts(classIndex) + rightCounts(classIndex) - classIndex += 1 - } - strategy.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) - } - } + if (metadata.isClassification) { + val leftTotalCount = leftNodeAgg.sum + val rightTotalCount = rightNodeAgg.sum - val totalCount = leftTotalCount + rightTotalCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val rootNodeCounts = new Array[Double](numClasses) + var classIndex = 0 + while (classIndex < numClasses) { + rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) + classIndex += 1 + } + metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) } + } - // Sum of count for each label - val leftRightCounts: Array[Double] = - leftCounts.zip(rightCounts).map { case (leftCount, rightCount) => - leftCount + rightCount - } + val totalCount = leftTotalCount + rightTotalCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } - def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { - case ((maxIndex, maxValue, currentIndex), currentValue) => - if (currentValue > maxValue) { - (currentIndex, currentValue, currentIndex + 1) - } else { - (maxIndex, maxValue, currentIndex + 1) - } - } - if (result._1 < 0) { - throw new RuntimeException("DecisionTree internal error:" + - " calculateGainForSplit failed in indexOfLargestArrayElement") - } - result._1 + // Sum of count for each label + val leftrightNodeAgg: Array[Double] = + leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => + leftCount + rightCount } - val predict = indexOfLargestArrayElement(leftRightCounts) - val prob = leftRightCounts(predict) / totalCount - - val leftImpurity = if (leftTotalCount == 0) { - topImpurity - } else { - strategy.impurity.calculate(leftCounts, leftTotalCount) + def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } } - val rightImpurity = if (rightTotalCount == 0) { - topImpurity - } else { - strategy.impurity.calculate(rightCounts, rightTotalCount) + if (result._1 < 0) { + throw new RuntimeException("DecisionTree internal error:" + + " calculateGainForSplit failed in indexOfLargestArrayElement") } + result._1 + } - val leftWeight = leftTotalCount / totalCount - val rightWeight = rightTotalCount / totalCount + val predict = indexOfLargestArrayElement(leftrightNodeAgg) + val prob = leftrightNodeAgg(predict) / totalCount - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + val leftImpurity = if (leftTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(leftNodeAgg, leftTotalCount) + } + val rightImpurity = if (rightTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(rightNodeAgg, rightTotalCount) + } - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + val leftWeight = leftTotalCount / totalCount + val rightWeight = rightTotalCount / totalCount - case Regression => - val leftCount = leftNodeAgg(featureIndex)(splitIndex)(0) - val leftSum = leftNodeAgg(featureIndex)(splitIndex)(1) - val leftSumSquares = leftNodeAgg(featureIndex)(splitIndex)(2) + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - val rightCount = rightNodeAgg(featureIndex)(splitIndex)(0) - val rightSum = rightNodeAgg(featureIndex)(splitIndex)(1) - val rightSumSquares = rightNodeAgg(featureIndex)(splitIndex)(2) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - strategy.impurity.calculate(count, sum, sumSquares) - } - } + } else { + // Regression - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity ,topImpurity, - Double.MinValue, leftSum / leftCount) + val leftCount = leftNodeAgg(0) + val leftSum = leftNodeAgg(1) + val leftSumSquares = leftNodeAgg(2) + + val rightCount = rightNodeAgg(0) + val rightSum = rightNodeAgg(1) + val rightSumSquares = rightNodeAgg(2) + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val count = leftCount + rightCount + val sum = leftSum + rightSum + val sumSquares = leftSumSquares + rightSumSquares + metadata.impurity.calculate(count, sum, sumSquares) } + } + + if (leftCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, + rightSum / rightCount) + } + if (rightCount == 0) { + return new InformationGainStats(0, topImpurity, topImpurity, + Double.MinValue, leftSum / leftCount) + } - val leftImpurity = strategy.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = strategy.impurity.calculate(rightCount, rightSum, rightSumSquares) + val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) + val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val leftWeight = leftCount.toDouble / (leftCount + rightCount) + val rightWeight = rightCount.toDouble / (leftCount + rightCount) - val gain = { - if (level > 0) { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } else { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + val predict = (leftSum + rightSum) / (leftCount + rightCount) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) } } @@ -1065,6 +974,19 @@ object DecisionTree extends Serializable with Logging { binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { + /** + * The input binData is indexed as (feature, bin, class). + * This computes cumulative sums over splits. + * Each (feature, class) pair is handled separately. + * Note: numSplits = numBins - 1. + * @param leftNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 0, ..., numSplits - 2) is set to be + * the cumulative sum (from left) over binData for bins 0, ..., i. + * @param rightNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 1, ..., numSplits - 1) is set to be + * the cumulative sum (from right) over binData for bins + * numBins - 1, ..., numBins - 1 - i. + */ def findAggForOrderedFeatureClassification( leftNodeAgg: Array[Array[Array[Double]]], rightNodeAgg: Array[Array[Array[Double]]], @@ -1169,45 +1091,32 @@ object DecisionTree extends Serializable with Logging { } } - strategy.algo match { - case Classification => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (isMulticlassClassificationWithCategoricalFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isSpaceSufficientForAllCategoricalSplits) { - findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - } - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - featureIndex += 1 - } - - (leftNodeAgg, rightNodeAgg) - case Regression => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) - featureIndex += 1 + if (metadata.isClassification) { + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } else { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) } - (leftNodeAgg, rightNodeAgg) + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) + } else { + // Regression + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) } } @@ -1225,8 +1134,9 @@ object DecisionTree extends Serializable with Logging { val numSplitsForFeature = getNumSplitsForFeature(featureIndex) var splitIndex = 0 while (splitIndex < numSplitsForFeature) { - gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex, - splitIndex, rightNodeAgg, nodeImpurity) + gains(featureIndex)(splitIndex) = + calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), + rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) splitIndex += 1 } featureIndex += 1 @@ -1238,18 +1148,14 @@ object DecisionTree extends Serializable with Logging { * Get the number of splits for a feature. */ def getNumSplitsForFeature(featureIndex: Int): Int = { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { + if (metadata.isContinuous(featureIndex)) { numBins - 1 } else { // Categorical feature - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits = - numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { - math.pow(2.0, featureCategories - 1).toInt - 1 + val featureCategories = metadata.featureArity(featureIndex) + if (metadata.isUnordered(featureIndex)) { + (1 << featureCategories - 1) - 1 } else { - // Ordered features featureCategories } } @@ -1308,29 +1214,29 @@ object DecisionTree extends Serializable with Logging { * Get bin data for one node. */ def getBinDataForNode(node: Int): Array[Double] = { - strategy.algo match { - case Classification => - if (isMulticlassClassificationWithCategoricalFeatures) { - val shift = numClasses * node * numBins * numFeatures - val rightChildShift = numClasses * numBins * numFeatures * numNodes - val binsForNode = { - val leftChildData - = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - val rightChildData - = binAggregates.slice(rightChildShift + shift, - rightChildShift + shift + numClasses * numBins * numFeatures) - leftChildData ++ rightChildData - } - binsForNode - } else { - val shift = numClasses * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - binsForNode + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + val shift = numClasses * node * numBins * numFeatures + val rightChildShift = numClasses * numBins * numFeatures * numNodes + val binsForNode = { + val leftChildData + = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + val rightChildData + = binAggregates.slice(rightChildShift + shift, + rightChildShift + shift + numClasses * numBins * numFeatures) + leftChildData ++ rightChildData } - case Regression => - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) binsForNode + } else { + val shift = numClasses * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + binsForNode + } + } else { + // Regression + val shift = 3 * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) + binsForNode } } @@ -1340,7 +1246,7 @@ object DecisionTree extends Serializable with Logging { // Iterating over all nodes at this level var node = 0 while (node < numNodes) { - val nodeImpurityIndex = math.pow(2, level).toInt - 1 + node + groupShift + val nodeImpurityIndex = (1 << level) - 1 + node + groupShift val binsForNode: Array[Double] = getBinDataForNode(node) logDebug("nodeImpurityIndex = " + nodeImpurityIndex) val parentNodeImpurity = parentImpurities(nodeImpurityIndex) @@ -1358,20 +1264,15 @@ object DecisionTree extends Serializable with Logging { * * @param numBins Number of bins = 1 + number of possible splits. */ - private def getElementsPerNode( - numFeatures: Int, - numBins: Int, - numClasses: Int, - isMulticlassClassificationWithCategoricalFeatures: Boolean, - algo: Algo): Int = { - algo match { - case Classification => - if (isMulticlassClassificationWithCategoricalFeatures) { - 2 * numClasses * numBins * numFeatures - } else { - numClasses * numBins * numFeatures - } - case Regression => 3 * numBins * numFeatures + private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + if (metadata.isClassification) { + if (metadata.isMulticlassWithCategoricalFeatures) { + 2 * metadata.numClasses * numBins * metadata.numFeatures + } else { + metadata.numClasses * numBins * metadata.numFeatures + } + } else { + 3 * numBins * metadata.numFeatures } } @@ -1390,16 +1291,15 @@ object DecisionTree extends Serializable with Logging { * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are math.pow(2, maxFeatureValue - 1) - 1 splits. + * There are (1 << maxFeatureValue - 1) - 1 splits. * (b) "ordered features" * For regression and binary classification, * and for multiclass classification with a high-arity feature, * there is one bin per category. * * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree - * @return A tuple of (splits,bins). + * @param metadata Learning and dataset metadata + * @return A tuple of (splits, bins). * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] * of size (numFeatures, numBins - 1). * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] @@ -1407,19 +1307,18 @@ object DecisionTree extends Serializable with Logging { */ protected[tree] def findSplitsBins( input: RDD[LabeledPoint], - strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = { + metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { val count = input.count() // Find the number of features by looking at the first sample val numFeatures = input.take(1)(0).features.size - val maxBins = strategy.maxBins + val maxBins = metadata.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt logDebug("numBins = " + numBins) - val isMulticlassClassification = strategy.isMulticlassClassification - logDebug("isMulticlassClassification = " + isMulticlassClassification) - + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) /* * Ensure numBins is always greater than the categories. For multiclass classification, @@ -1431,13 +1330,12 @@ object DecisionTree extends Serializable with Logging { * by the number of training examples. * TODO: Allow this case, where we simply will know nothing about some categories. */ - if (strategy.categoricalFeaturesInfo.size > 0) { - val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 + if (metadata.featureArity.size > 0) { + val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + "in categorical features") } - // Calculate the number of sample for approximate quantile calculation. val requiredSamples = numBins*numBins val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 @@ -1451,7 +1349,7 @@ object DecisionTree extends Serializable with Logging { val stride: Double = numSamples.toDouble / numBins logDebug("stride = " + stride) - strategy.quantileCalculationStrategy match { + metadata.quantileStrategy match { case Sort => val splits = Array.ofDim[Split](numFeatures, numBins - 1) val bins = Array.ofDim[Bin](numFeatures, numBins) @@ -1462,7 +1360,7 @@ object DecisionTree extends Serializable with Logging { var featureIndex = 0 while (featureIndex < numFeatures) { // Check whether the feature is continuous. - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted val stride: Double = numSamples.toDouble / numBins @@ -1475,18 +1373,14 @@ object DecisionTree extends Serializable with Logging { splits(featureIndex)(index) = split } } else { // Categorical feature - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + val featureCategories = metadata.featureArity(featureIndex) // Use different bin/split calculation strategy for categorical features in multiclass // classification that satisfy the space constraint. - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - if (isUnorderedFeature) { + if (metadata.isUnordered(featureIndex)) { // 2^(maxFeatureValue- 1) - 1 combinations var index = 0 - while (index < math.pow(2.0, featureCategories - 1).toInt - 1) { + while (index < (1 << featureCategories - 1) - 1) { val categories: List[Double] = extractMultiClassCategories(index + 1, featureCategories) splits(featureIndex)(index) @@ -1516,7 +1410,7 @@ object DecisionTree extends Serializable with Logging { * centroidForCategories is a mapping: category (for the given feature) --> centroid */ val centroidForCategories = { - if (isMulticlassClassification) { + if (isMulticlass) { // For categorical variables in multiclass classification, // each bin is a category. The bins are sorted and they // are ordered by calculating the impurity of their corresponding labels. @@ -1524,7 +1418,7 @@ object DecisionTree extends Serializable with Logging { .groupBy(_._1) .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, strategy.impurity.calculate(x._2, x._2.sum))) + .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) } else { // regression or binary classification // For categorical variables in regression and binary classification, // each bin is a category. The bins are sorted and they @@ -1576,7 +1470,7 @@ object DecisionTree extends Serializable with Logging { // Find all bins. featureIndex = 0 while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { // Bins for categorical variables are already assigned. bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) @@ -1590,7 +1484,7 @@ object DecisionTree extends Serializable with Logging { } featureIndex += 1 } - (splits,bins) + (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") case ApproxHist => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala new file mode 100644 index 0000000000000..d9eda354dc986 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -0,0 +1,101 @@ +/* + * 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.mllib.tree.impl + +import scala.collection.mutable + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.rdd.RDD + + +/** + * Learning and dataset metadata for DecisionTree. + * + * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. + * For regression: fixed at 0 (no meaning). + * @param featureArity Map: categorical feature index --> arity. + * I.e., the feature takes values in {0, ..., arity - 1}. + */ +private[tree] class DecisionTreeMetadata( + val numFeatures: Int, + val numExamples: Long, + val numClasses: Int, + val maxBins: Int, + val featureArity: Map[Int, Int], + val unorderedFeatures: Set[Int], + val impurity: Impurity, + val quantileStrategy: QuantileStrategy) extends Serializable { + + def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) + + def isClassification: Boolean = numClasses >= 2 + + def isMulticlass: Boolean = numClasses > 2 + + def isMulticlassWithCategoricalFeatures: Boolean = isMulticlass && (featureArity.size > 0) + + def isCategorical(featureIndex: Int): Boolean = featureArity.contains(featureIndex) + + def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + +} + +private[tree] object DecisionTreeMetadata { + + def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { + + val numFeatures = input.take(1)(0).features.size + val numExamples = input.count() + val numClasses = strategy.algo match { + case Classification => strategy.numClassesForClassification + case Regression => 0 + } + + val maxBins = math.min(strategy.maxBins, numExamples).toInt + val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + + val unorderedFeatures = new mutable.HashSet[Int]() + if (numClasses > 2) { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + if (k - 1 < log2MaxBinsp1) { + // Note: The above check is equivalent to checking: + // numUnorderedBins = (1 << k - 1) - 1 < maxBins + unorderedFeatures.add(f) + } else { + // TODO: Allow this case, where we simply will know nothing about some categories? + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + } else { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + strategy.impurity, strategy.quantileCalculationStrategy) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index ccac1031fd9d9..170e43e222083 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -18,7 +18,6 @@ package org.apache.spark.mllib.tree.impl import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.model.Bin import org.apache.spark.rdd.RDD @@ -48,50 +47,35 @@ private[tree] object TreePoint { * Convert an input dataset into its TreePoint representation, * binning feature values in preparation for DecisionTree training. * @param input Input dataset. - * @param strategy DecisionTree training info, used for dataset metadata. * @param bins Bins for features, of size (numFeatures, numBins). + * @param metadata Learning and dataset metadata * @return TreePoint dataset representation */ def convertToTreeRDD( input: RDD[LabeledPoint], - strategy: Strategy, - bins: Array[Array[Bin]]): RDD[TreePoint] = { + bins: Array[Array[Bin]], + metadata: DecisionTreeMetadata): RDD[TreePoint] = { input.map { x => - TreePoint.labeledPointToTreePoint(x, strategy.isMulticlassClassification, bins, - strategy.categoricalFeaturesInfo) + TreePoint.labeledPointToTreePoint(x, bins, metadata) } } /** * Convert one LabeledPoint into its TreePoint representation. * @param bins Bins for features, of size (numFeatures, numBins). - * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, - isMulticlassClassification: Boolean, bins: Array[Array[Bin]], - categoricalFeaturesInfo: Map[Int, Int]): TreePoint = { + metadata: DecisionTreeMetadata): TreePoint = { val numFeatures = labeledPoint.features.size val numBins = bins(0).size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { - val featureInfo = categoricalFeaturesInfo.get(featureIndex) - val isFeatureContinuous = featureInfo.isEmpty - if (isFeatureContinuous) { - arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, false, - bins, categoricalFeaturesInfo) - } else { - val featureCategories = featureInfo.get - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, - isUnorderedFeature, bins, categoricalFeaturesInfo) - } + arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), + metadata.isUnordered(featureIndex), bins, metadata.featureArity) featureIndex += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index c89c1e371a40e..af35d88f713e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,15 +20,25 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. For a continuous - * feature, a bin is determined by a low and a high "split". For a categorical feature, - * the a bin is determined using a single label value (category). + * Used for "binning" the features bins for faster best split calculation. + * + * For a continuous feature, the bin is determined by a low and a high split, + * where an example with featureValue falls into the bin s.t. + * lowSplit.threshold < featureValue <= highSplit.threshold. + * + * For ordered categorical features, there is a 1-1-1 correspondence between + * bins, splits, and feature values. The bin is determined by category/feature value. + * However, the bins are not necessarily ordered by feature value; + * they are ordered using impurity. + * For unordered categorical features, there is a 1-1 correspondence between bins, splits, + * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be * accepted in the bin * @param featureType type of feature -- categorical or continuous - * @param category categorical label value accepted in the bin for binary classification + * @param category categorical label value accepted in the bin for ordered features */ private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 3d3406b5d5f22..0594fd0749d21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -39,7 +39,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @return Double prediction from the trained model */ def predict(features: Vector): Double = { - topNode.predictIfLeaf(features) + topNode.predict(features) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala deleted file mode 100644 index 2deaf4ae8dcab..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.tree.model - -/** - * Filter specifying a split and type of comparison to be applied on features - * @param split split specifying the feature index, type and threshold - * @param comparison integer specifying <,=,> - */ -private[tree] case class Filter(split: Split, comparison: Int) { - // Comparison -1,0,1 signifies <.=,> - override def toString = " split = " + split + "comparison = " + comparison -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 944f11c2c2e4f..0eee6262781c1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -69,24 +69,24 @@ class Node ( /** * predict value if node is not leaf - * @param feature feature value + * @param features feature value * @return predicted value */ - def predictIfLeaf(feature: Vector) : Double = { + def predict(features: Vector) : Double = { if (isLeaf) { predict } else{ if (split.get.featureType == Continuous) { - if (feature(split.get.feature) <= split.get.threshold) { - leftNode.get.predictIfLeaf(feature) + if (features(split.get.feature) <= split.get.threshold) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } else { - if (split.get.categories.contains(feature(split.get.feature))) { - leftNode.get.predictIfLeaf(feature) + if (split.get.categories.contains(features(split.get.feature))) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index d7ffd386c05ee..50fb48b40de3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -24,9 +24,10 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType * :: DeveloperApi :: * Split applied to a feature * @param feature feature index - * @param threshold threshold for continuous feature + * @param threshold Threshold for continuous feature. + * Split left if feature <= threshold, else right. * @param featureType type of feature -- categorical or continuous - * @param categories accepted values for categorical variables + * @param categories Split left if categorical feature value is in this set, else right. */ @DeveloperApi case class Split( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index a5c49a38dc08f..2f36fd907772c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -23,10 +23,10 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} -import org.apache.spark.mllib.tree.impl.TreePoint +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Filter, Split} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.regression.LabeledPoint @@ -64,7 +64,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -82,7 +83,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -162,7 +164,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // Check splits. @@ -279,7 +282,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -373,7 +377,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // 2^10 - 1 > 100, so categorical variables will be ordered @@ -428,10 +433,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -456,10 +462,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -495,7 +502,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -503,9 +511,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -518,7 +526,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -526,9 +535,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -542,7 +551,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -550,9 +560,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -566,7 +576,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -574,9 +585,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -590,7 +601,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -598,14 +610,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val leftFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()), -1) - val rightFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()) ,1) - val filters = Array[List[Filter]](List(), List(leftFilter), List(rightFilter)) + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val nodes: Array[Node] = new Array[Node](7) + nodes(0) = modelOneNode.topNode + nodes(0).leftNode = None + nodes(0).rightNode = None + val parentImpurities = Array(0.5, 0.5, 0.5) // Single group second level tree construction. - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, filters, + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, nodes, splits, bins, 10) assert(bestSplits.length === 2) assert(bestSplits(0)._2.gain > 0) @@ -613,8 +630,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, - filters, splits, bins, 0) + val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, + nodes, splits, bins, 0) assert(bestSplitsWithGroups.length === 2) assert(bestSplitsWithGroups(0)._2.gain > 0) assert(bestSplitsWithGroups(1)._2.gain > 0) @@ -629,19 +646,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) } - } test("stump with categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -657,11 +674,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) @@ -688,20 +705,22 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for multiclass classification, with just enough bins") { val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + numClassesForClassification = 3, maxBins = maxBins, + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -716,18 +735,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with continuous variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -741,18 +761,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with continuous + categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -765,14 +786,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for ordered multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 From d411f4190252546b0ea99c1934efd5e5f84be50c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 17 Aug 2014 15:48:39 -0700 Subject: [PATCH 016/192] SPARK-2881: Upgrade to Snappy 1.0.5.3 to avoid SPARK-2881. This version of Snappy was released with a backported fix specifically for Spark. This fixes an issue where names collide in the snappy .so file when users are submitting jobs as different users on the same cluster. Author: Patrick Wendell Closes #1999 from pwendell/snappy-upgrade and squashes the following commits: 38974ff [Patrick Wendell] SPARK-2881: Upgrade to Snappy 1.0.5.3 to avoid SPARK-2881. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c87f776bda659..9e5217e294681 100644 --- a/pom.xml +++ b/pom.xml @@ -315,7 +315,7 @@ org.xerial.snappy snappy-java - 1.0.5 + 1.0.5.3 net.jpountz.lz4 From c6a0091ea401e0bec58d7607eb42be89cc090868 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 17 Aug 2014 18:10:45 -0700 Subject: [PATCH 017/192] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" Revert #1891 due to issues with hadoop 1 compatibility. Author: Michael Armbrust Closes #2007 from marmbrus/revert1891 and squashes the following commits: 68706c0 [Michael Armbrust] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" (cherry picked from commit 5ecb08ea063166564178885b7515abef0d76eecb) Signed-off-by: Michael Armbrust --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index c16a7d3661c66..b092f42372171 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -26,8 +26,6 @@ import jline.{ConsoleReader, History} import org.apache.commons.lang.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} @@ -118,17 +116,13 @@ private[hive] object SparkSQLCLIDriver { SessionState.start(sessionState) // Clean up after we exit - /** - * This should be executed before shutdown hook of - * FileSystem to avoid race condition of FileSystem operation - */ - ShutdownHookManager.get.addShutdownHook( + Runtime.getRuntime.addShutdownHook( new Thread() { override def run() { SparkSQLEnv.stop() } } - , FileSystem.SHUTDOWN_HOOK_PRIORITY - 1) + ) // "-h" option has been passed, so connect to Hive thrift server. if (sessionState.getHost != null) { From 4f776dfab726f54c948a83a7157b958903c15ecf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 17 Aug 2014 19:00:38 -0700 Subject: [PATCH 018/192] [SQL] Improve debug logging and toStrings. Author: Michael Armbrust Closes #2004 from marmbrus/codgenDebugging and squashes the following commits: b7a7e41 [Michael Armbrust] Improve debug logging and toStrings. (cherry picked from commit bfa09b01d7eddc572cd22ca2e418a735b4ccc826) Signed-off-by: Michael Armbrust --- .../expressions/codegen/CodeGenerator.scala | 21 +++++++++++++++++-- .../catalyst/expressions/nullFunctions.scala | 2 ++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 5b398695bf560..de2d67ce82ff1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -78,7 +78,12 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin .build( new CacheLoader[InType, OutType]() { override def load(in: InType): OutType = globalLock.synchronized { - create(in) + val startTime = System.nanoTime() + val result = create(in) + val endTime = System.nanoTime() + def timeMs = (endTime - startTime).toDouble / 1000000 + logInfo(s"Code generated expression $in in $timeMs ms") + result } }) @@ -413,7 +418,19 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin """.children } - EvaluatedExpression(code, nullTerm, primitiveTerm, objectTerm) + // Only inject debugging code if debugging is turned on. + val debugCode = + if (log.isDebugEnabled) { + val localLogger = log + val localLoggerTree = reify { localLogger } + q""" + $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + """ :: Nil + } else { + Nil + } + + EvaluatedExpression(code ++ debugCode, nullTerm, primitiveTerm, objectTerm) } protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index ce6d99c911ab3..e88c5d4fa178a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -60,6 +60,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def eval(input: Row): Any = { child.eval(input) == null } + + override def toString = s"IS NULL $child" } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { From 826356725ffb3189180f7879d3f9c449924785f3 Mon Sep 17 00:00:00 2001 From: Chris Fregly Date: Sun, 17 Aug 2014 19:33:15 -0700 Subject: [PATCH 019/192] [SPARK-1981] updated streaming-kinesis.md fixed markup, separated out sections more-clearly, more thorough explanations Author: Chris Fregly Closes #1757 from cfregly/master and squashes the following commits: 9b1c71a [Chris Fregly] better explained why spark checkpoints are disabled in the example (due to no stateful operations being used) 0f37061 [Chris Fregly] SPARK-1981: (Kinesis streaming support) updated streaming-kinesis.md 862df67 [Chris Fregly] Merge remote-tracking branch 'upstream/master' 8e1ae2e [Chris Fregly] Merge remote-tracking branch 'upstream/master' 4774581 [Chris Fregly] updated docs, renamed retry to retryRandom to be more clear, removed retries around store() method 0393795 [Chris Fregly] moved Kinesis examples out of examples/ and back into extras/kinesis-asl 691a6be [Chris Fregly] fixed tests and formatting, fixed a bug with JavaKinesisWordCount during union of streams 0e1c67b [Chris Fregly] Merge remote-tracking branch 'upstream/master' 74e5c7c [Chris Fregly] updated per TD's feedback. simplified examples, updated docs e33cbeb [Chris Fregly] Merge remote-tracking branch 'upstream/master' bf614e9 [Chris Fregly] per matei's feedback: moved the kinesis examples into the examples/ dir d17ca6d [Chris Fregly] per TD's feedback: updated docs, simplified the KinesisUtils api 912640c [Chris Fregly] changed the foundKinesis class to be a publically-avail class db3eefd [Chris Fregly] Merge remote-tracking branch 'upstream/master' 21de67f [Chris Fregly] Merge remote-tracking branch 'upstream/master' 6c39561 [Chris Fregly] parameterized the versions of the aws java sdk and kinesis client 338997e [Chris Fregly] improve build docs for kinesis 828f8ae [Chris Fregly] more cleanup e7c8978 [Chris Fregly] Merge remote-tracking branch 'upstream/master' cd68c0d [Chris Fregly] fixed typos and backward compatibility d18e680 [Chris Fregly] Merge remote-tracking branch 'upstream/master' b3b0ff1 [Chris Fregly] [SPARK-1981] Add AWS Kinesis streaming support (cherry picked from commit 99243288b049f4a4fb4ba0505ea2310be5eb4bd2) Signed-off-by: Tathagata Das --- docs/streaming-kinesis.md | 97 ++++++++++++++++++++------------------- 1 file changed, 49 insertions(+), 48 deletions(-) diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md index 801c905c88df8..16ad3222105a2 100644 --- a/docs/streaming-kinesis.md +++ b/docs/streaming-kinesis.md @@ -3,56 +3,57 @@ layout: global title: Spark Streaming Kinesis Receiver --- -### Kinesis -Build notes: -
  • Spark supports a Kinesis Streaming Receiver which is not included in the default build due to licensing restrictions.
  • -
  • _**Note that by embedding this library you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • -
  • The Spark Kinesis Streaming Receiver source code, examples, tests, and artifacts live in $SPARK_HOME/extras/kinesis-asl.
  • -
  • To build with Kinesis, you must run the maven or sbt builds with -Pkinesis-asl`.
  • -
  • Applications will need to link to the 'spark-streaming-kinesis-asl` artifact.
  • +## Kinesis +###Design +
  • The KinesisReceiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License.
  • +
  • The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases.
  • +
  • The KCL uses DynamoDB to maintain all state. A DynamoDB table is created in the us-east-1 region (regardless of Kinesis stream region) during KCL initialization for each Kinesis application name.
  • +
  • A single KinesisReceiver can process many shards of a stream by spinning up multiple KinesisRecordProcessor threads.
  • +
  • You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread.
  • +
  • Horizontal scaling is achieved by autoscaling additional KinesisReceiver (separate processes) or spinning up new KinesisRecordProcessor threads within each KinesisReceiver - up to the number of current shards for a given stream, of course. Don't forget to autoscale back down!
  • -Kinesis examples notes: -
  • To build the Kinesis examples, you must run the maven or sbt builds with -Pkinesis-asl`.
  • -
  • These examples automatically determine the number of local threads and KinesisReceivers to spin up based on the number of shards for the stream.
  • -
  • KinesisWordCountProducerASL will generate random data to put onto the Kinesis stream for testing.
  • -
  • Checkpointing is disabled (no checkpoint dir is set). The examples as written will not recover from a driver failure.
  • +### Build +
  • Spark supports a Streaming KinesisReceiver, but it is not included in the default build due to Amazon Software Licensing (ASL) restrictions.
  • +
  • To build with the Kinesis Streaming Receiver and supporting ASL-licensed code, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • All KinesisReceiver-related code, examples, tests, and artifacts live in **$SPARK_HOME/extras/kinesis-asl/**.
  • +
  • Kinesis-based Spark Applications will need to link to the **spark-streaming-kinesis-asl** artifact that is built when **-Pkinesis-asl** is specified.
  • +
  • _**Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • -Deployment and runtime notes: -
  • A single KinesisReceiver can process many shards of a stream.
  • -
  • Each shard of a stream is processed by one or more KinesisReceiver's managed by the Kinesis Client Library (KCL) Worker.
  • -
  • You never need more KinesisReceivers than the number of shards in your stream.
  • -
  • You can horizontally scale the receiving by creating more KinesisReceiver/DStreams (up to the number of shards for a given stream)
  • -
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the Kinesis Client Library.
  • -
  • This code uses the DefaultAWSCredentialsProviderChain and searches for credentials in the following order of precedence:
    - 1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    - 2) Java System Properties - aws.accessKeyId and aws.secretKey
    - 3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    - 4) Instance profile credentials - delivered through the Amazon EC2 metadata service
    -
  • -
  • You need to setup a Kinesis stream with 1 or more shards per the following:
    - http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • -
  • Valid Kinesis endpoint urls can be found here: Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • -
  • When you first start up the KinesisReceiver, the Kinesis Client Library (KCL) needs ~30s to establish connectivity with the AWS Kinesis service, -retrieve any checkpoint data, and negotiate with other KCL's reading from the same stream.
  • -
  • Be careful when changing the app name. Kinesis maintains a mapping table in DynamoDB based on this app name (http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization). -Changing the app name could lead to Kinesis errors as only 1 logical application can process a stream. In order to start fresh, -it's always best to delete the DynamoDB table that matches your app name. This DynamoDB table lives in us-east-1 regardless of the Kinesis endpoint URL.
  • +###Example +
  • To build the Kinesis example, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • You need to setup a Kinesis stream at one of the valid Kinesis endpoints with 1 or more shards per the following: http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • +
  • Valid Kinesis endpoints can be found here: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • +
  • When running **locally**, the example automatically determines the number of threads and KinesisReceivers to spin up based on the number of shards configured for the stream. Therefore, **local[n]** is not needed when starting the example as with other streaming examples.
  • +
  • While this example could use a single KinesisReceiver which spins up multiple KinesisRecordProcessor threads to process multiple shards, I wanted to demonstrate unioning multiple KinesisReceivers as a single DStream. (It's a bit confusing in local mode.)
  • +
  • **KinesisWordCountProducerASL** is provided to generate random records into the Kinesis stream for testing.
  • +
  • The example has been configured to immediately replicate incoming stream data to another node by using (StorageLevel.MEMORY_AND_DISK_2) +
  • Spark checkpointing is disabled because the example does not use any stateful or window-based DStream operations such as updateStateByKey and reduceByWindow. If those operations are introduced, you would need to enable checkpointing or risk losing data in the case of a failure.
  • +
  • Kinesis checkpointing is enabled. This means that the example will recover from a Kinesis failure.
  • +
  • The example uses InitialPositionInStream.LATEST strategy to pull from the latest tip of the stream if no Kinesis checkpoint info exists.
  • +
  • In our example, **KinesisWordCount** is the Kinesis application name for both the Scala and Java versions. The use of this application name is described next.
  • -Failure recovery notes: -
  • The combination of Spark Streaming and Kinesis creates 3 different checkpoints as follows:
    - 1) RDD data checkpoint (Spark Streaming) - frequency is configurable with DStream.checkpoint(Duration)
    - 2) RDD metadata checkpoint (Spark Streaming) - frequency is every DStream batch
    - 3) Kinesis checkpointing (Kinesis) - frequency is controlled by the developer calling ICheckpointer.checkpoint() directly
    +###Deployment and Runtime +
  • A Kinesis application name must be unique for a given account and region.
  • +
  • A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization
  • +
  • This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL.
  • +
  • Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream.
  • +
  • If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch.
  • +
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the KCL.
  • +
  • The KinesisReceiver uses the DefaultAWSCredentialsProviderChain for AWS credentials which searches for credentials in the following order of precedence:
    +1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    +2) Java System Properties - aws.accessKeyId and aws.secretKey
    +3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    +4) Instance profile credentials - delivered through the Amazon EC2 metadata service
  • -
  • Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling
  • -
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last checkpoint sequence number recorded per shard.
  • -
  • If no checkpoint info exists, the worker will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) -or from the tip/latest (InitialPostitionInStream.LATEST). This is configurable.
  • -
  • When pulling from the stream tip (InitialPositionInStream.LATEST), only new stream data will be picked up after the KinesisReceiver starts.
  • -
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running.
  • -
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data -depending on the checkpoint frequency.
  • -
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records depending on the checkpoint frequency.
  • + +###Fault-Tolerance +
  • The combination of Spark Streaming and Kinesis creates 2 different checkpoints that may occur at different intervals.
  • +
  • Checkpointing too frequently against Kinesis will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random backoff retry strategy.
  • +
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last Kinesis checkpoint sequence number recorded per shard (stored in the DynamoDB table).
  • +
  • If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable.
  • +
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored.)
  • +
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data.
  • +
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency.
  • Record processing should be idempotent when possible.
  • -
  • Failed or latent KinesisReceivers will be detected and automatically shutdown/load-balanced by the KCL.
  • -
  • If possible, explicitly shutdown the worker if a failure occurs in order to trigger the final checkpoint.
  • +
  • A failed or latent KinesisRecordProcessor within the KinesisReceiver will be detected and automatically restarted by the KCL.
  • +
  • If possible, the KinesisReceiver should be shutdown cleanly in order to trigger a final checkpoint of all KinesisRecordProcessors to avoid duplicate record processing.
  • \ No newline at end of file From 8438daf2c2a04e48465fc2681d142ca5a6dec747 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 17 Aug 2014 20:53:18 -0700 Subject: [PATCH 020/192] [SPARK-3087][MLLIB] fix col indexing bug in chi-square and add a check for number of distinct values There is a bug determining the column index. dorx Author: Xiangrui Meng Closes #1997 from mengxr/chisq-index and squashes the following commits: 8fc2ab2 [Xiangrui Meng] fix col indexing bug and add a check for number of distinct values (cherry picked from commit c77f40668fbb5b8bca9a9b25c039895cb7a4a80c) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/stat/Statistics.scala | 2 +- .../spark/mllib/stat/test/ChiSqTest.scala | 37 +++++++++++++++---- .../mllib/stat/HypothesisTestSuite.scala | 37 ++++++++++++++----- 3 files changed, 59 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index 3cf1028fbc725..3cf4e807b4cf7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -155,7 +155,7 @@ object Statistics { * :: Experimental :: * Conduct Pearson's independence test for every feature against the label across the input RDD. * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. + * the chi-squared statistic is computed. All label and feature values must be categorical. * * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. * Real-valued features will be treated as categorical for each distinct value. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 215de95db5113..0089419c2c5d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -20,11 +20,13 @@ package org.apache.spark.mllib.stat.test import breeze.linalg.{DenseMatrix => BDM} import cern.jet.stat.Probability.chiSquareComplemented -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD +import scala.collection.mutable + /** * Conduct the chi-squared test for the input RDDs using the specified method. * Goodness-of-fit test is conducted on two `Vectors`, whereas test of independence is conducted @@ -75,21 +77,42 @@ private[stat] object ChiSqTest extends Logging { */ def chiSquaredFeatures(data: RDD[LabeledPoint], methodName: String = PEARSON.name): Array[ChiSqTestResult] = { + val maxCategories = 10000 val numCols = data.first().features.size val results = new Array[ChiSqTestResult](numCols) var labels: Map[Double, Int] = null - // At most 100 columns at a time - val batchSize = 100 + // at most 1000 columns at a time + val batchSize = 1000 var batch = 0 while (batch * batchSize < numCols) { // The following block of code can be cleaned up and made public as // chiSquared(data: RDD[(V1, V2)]) val startCol = batch * batchSize val endCol = startCol + math.min(batchSize, numCols - startCol) - val pairCounts = data.flatMap { p => - // assume dense vectors - p.features.toArray.slice(startCol, endCol).zipWithIndex.map { case (feature, col) => - (col, feature, p.label) + val pairCounts = data.mapPartitions { iter => + val distinctLabels = mutable.HashSet.empty[Double] + val allDistinctFeatures: Map[Int, mutable.HashSet[Double]] = + Map((startCol until endCol).map(col => (col, mutable.HashSet.empty[Double])): _*) + var i = 1 + iter.flatMap { case LabeledPoint(label, features) => + if (i % 1000 == 0) { + if (distinctLabels.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct label values.") + } + allDistinctFeatures.foreach { case (col, distinctFeatures) => + if (distinctFeatures.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct values in column $col.") + } + } + } + i += 1 + distinctLabels += label + features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + allDistinctFeatures(col) += feature + (col, feature, label) + } } }.countByValue() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala index 5bd0521298c14..6de3840b3f198 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.mllib.stat +import java.util.Random + import org.scalatest.FunSuite +import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.{DenseVector, Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.test.ChiSqTest @@ -107,12 +110,13 @@ class HypothesisTestSuite extends FunSuite with LocalSparkContext { // labels: 1.0 (2 / 6), 0.0 (4 / 6) // feature1: 0.5 (1 / 6), 1.5 (2 / 6), 3.5 (3 / 6) // feature2: 10.0 (1 / 6), 20.0 (1 / 6), 30.0 (2 / 6), 40.0 (2 / 6) - val data = Array(new LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), - new LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), - new LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), - new LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), - new LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), - new LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) + val data = Seq( + LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), + LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), + LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), + LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) for (numParts <- List(2, 4, 6, 8)) { val chi = Statistics.chiSqTest(sc.parallelize(data, numParts)) val feature1 = chi(0) @@ -130,10 +134,25 @@ class HypothesisTestSuite extends FunSuite with LocalSparkContext { } // Test that the right number of results is returned - val numCols = 321 - val sparseData = Array(new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), - new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((200, 1.0))))) + val numCols = 1001 + val sparseData = Array( + new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), + new LabeledPoint(0.1, Vectors.sparse(numCols, Seq((200, 1.0))))) val chi = Statistics.chiSqTest(sc.parallelize(sparseData)) assert(chi.size === numCols) + assert(chi(1000) != null) // SPARK-3087 + + // Detect continous features or labels + val random = new Random(11L) + val continuousLabel = + Seq.fill(100000)(LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousLabel, 2)) + } + val continuousFeature = + Seq.fill(100000)(LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousFeature, 2)) + } } } From a5ae720745d744ec29741b49d2d362f362d53fa4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 17 Aug 2014 22:29:58 -0700 Subject: [PATCH 021/192] SPARK-2884: Create binary builds in parallel with release script. --- dev/create-release/create-release.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 1867cf4ec46ca..28f26d2368254 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -117,12 +117,13 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & make_binary_release "hadoop2-without-hive" \ - "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +wait # Copy data echo "Copying release tarballs" From 0506539b0e853d474183078814fb0f550bfbbd67 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 17 Aug 2014 22:39:06 -0700 Subject: [PATCH 022/192] SPARK-2900. aggregate inputBytes per stage Author: Sandy Ryza Closes #1826 from sryza/sandy-spark-2900 and squashes the following commits: 43f9091 [Sandy Ryza] SPARK-2900 (cherry picked from commit df652ea02a3e42d987419308ef14874300347373) Signed-off-by: Patrick Wendell --- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 6 ++++++ .../apache/spark/ui/jobs/JobProgressListenerSuite.scala | 9 ++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index a3e9566832d06..74cd637d88155 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -200,6 +200,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.shuffleReadBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta + val inputBytesDelta = + (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) + stageData.inputBytes += inputBytesDelta + execSummary.inputBytes += inputBytesDelta + val diskSpillDelta = taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) stageData.diskBytesSpilled += diskSpillDelta diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index f5ba31c309277..147ec0bc52e39 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.Matchers import org.apache.spark._ import org.apache.spark.{LocalSparkContext, SparkConf, Success} -import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -150,6 +150,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.executorRunTime = base + 4 taskMetrics.diskBytesSpilled = base + 5 taskMetrics.memoryBytesSpilled = base + 6 + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.bytesRead = base + 7 taskMetrics } @@ -182,6 +185,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(stage1Data.diskBytesSpilled == 205) assert(stage0Data.memoryBytesSpilled == 112) assert(stage1Data.memoryBytesSpilled == 206) + assert(stage0Data.inputBytes == 114) + assert(stage1Data.inputBytes == 207) assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get .totalBlocksFetched == 2) assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get @@ -208,6 +213,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(stage1Data.diskBytesSpilled == 610) assert(stage0Data.memoryBytesSpilled == 412) assert(stage1Data.memoryBytesSpilled == 612) + assert(stage0Data.inputBytes == 414) + assert(stage1Data.inputBytes == 614) assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get .totalBlocksFetched == 302) assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get From 708cde99a142c90f5a06c7aa326b622d80022e3d Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 17 Aug 2014 23:29:44 -0700 Subject: [PATCH 023/192] [SPARK-3097][MLlib] Word2Vec performance improvement mengxr Please review the code. Adding weights in reduceByKey soon. Only output model entry for words appeared in the partition before merging and use reduceByKey to combine model. In general, this implementation is 30s or so faster than implementation using big array. Author: Liquan Pei Closes #1932 from Ishiihara/Word2Vec-improve2 and squashes the following commits: d5377a9 [Liquan Pei] use syn0Global and syn1Global to represent model cad2011 [Liquan Pei] bug fix for synModify array out of bound 083aa66 [Liquan Pei] update synGlobal in place and reduce synOut size 9075e1c [Liquan Pei] combine syn0Global and syn1Global to synGlobal aa2ab36 [Liquan Pei] use reduceByKey to combine models (cherry picked from commit 3c8fa505900ac158d57de36f6b0fd6da05f8893b) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/feature/Word2Vec.scala | 50 +++++++++++++------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index ecd49ea2ff533..d2ae62b482aff 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -34,6 +34,7 @@ import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** * Entry in vocabulary @@ -287,11 +288,12 @@ class Word2Vec extends Serializable with Logging { var syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) var syn1Global = new Array[Float](vocabSize * vectorSize) - var alpha = startingAlpha for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) + val syn0Modify = new Array[Int](vocabSize) + val syn1Modify = new Array[Int](vocabSize) val model = iter.foldLeft((syn0Global, syn1Global, 0, 0)) { case ((syn0, syn1, lastWordCount, wordCount), sentence) => var lwc = lastWordCount @@ -321,7 +323,8 @@ class Word2Vec extends Serializable with Logging { // Hierarchical softmax var d = 0 while (d < bcVocab.value(word).codeLen) { - val l2 = bcVocab.value(word).point(d) * vectorSize + val inner = bcVocab.value(word).point(d) + val l2 = inner * vectorSize // Propagate hidden -> output var f = blas.sdot(vectorSize, syn0, l1, 1, syn1, l2, 1) if (f > -MAX_EXP && f < MAX_EXP) { @@ -330,10 +333,12 @@ class Word2Vec extends Serializable with Logging { val g = ((1 - bcVocab.value(word).code(d) - f) * alpha).toFloat blas.saxpy(vectorSize, g, syn1, l2, 1, neu1e, 0, 1) blas.saxpy(vectorSize, g, syn0, l1, 1, syn1, l2, 1) + syn1Modify(inner) += 1 } d += 1 } blas.saxpy(vectorSize, 1.0f, neu1e, 0, 1, syn0, l1, 1) + syn0Modify(lastWord) += 1 } } a += 1 @@ -342,21 +347,36 @@ class Word2Vec extends Serializable with Logging { } (syn0, syn1, lwc, wc) } - Iterator(model) + val syn0Local = model._1 + val syn1Local = model._2 + val synOut = new PrimitiveKeyOpenHashMap[Int, Array[Float]](vocabSize * 2) + var index = 0 + while(index < vocabSize) { + if (syn0Modify(index) != 0) { + synOut.update(index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + if (syn1Modify(index) != 0) { + synOut.update(index + vocabSize, + syn1Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + index += 1 + } + Iterator(synOut) } - val (aggSyn0, aggSyn1, _, _) = - partial.treeReduce { case ((syn0_1, syn1_1, lwc_1, wc_1), (syn0_2, syn1_2, lwc_2, wc_2)) => - val n = syn0_1.length - val weight1 = 1.0f * wc_1 / (wc_1 + wc_2) - val weight2 = 1.0f * wc_2 / (wc_1 + wc_2) - blas.sscal(n, weight1, syn0_1, 1) - blas.sscal(n, weight1, syn1_1, 1) - blas.saxpy(n, weight2, syn0_2, 1, syn0_1, 1) - blas.saxpy(n, weight2, syn1_2, 1, syn1_1, 1) - (syn0_1, syn1_1, lwc_1 + lwc_2, wc_1 + wc_2) + val synAgg = partial.flatMap(x => x).reduceByKey { case (v1, v2) => + blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) + v1 + }.collect() + var i = 0 + while (i < synAgg.length) { + val index = synAgg(i)._1 + if (index < vocabSize) { + Array.copy(synAgg(i)._2, 0, syn0Global, index * vectorSize, vectorSize) + } else { + Array.copy(synAgg(i)._2, 0, syn1Global, (index - vocabSize) * vectorSize, vectorSize) } - syn0Global = aggSyn0 - syn1Global = aggSyn1 + i += 1 + } } newSentences.unpersist() From 518258f1ba4d79a72e1a97ebebb1b51cd392c503 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 17 Aug 2014 23:30:47 -0700 Subject: [PATCH 024/192] [SPARK-2842][MLlib]Word2Vec documentation mengxr Documentation for Word2Vec Author: Liquan Pei Closes #2003 from Ishiihara/Word2Vec-doc and squashes the following commits: 4ff11d4 [Liquan Pei] minor fix 8d7458f [Liquan Pei] code reformat 6df0dcb [Liquan Pei] add Word2Vec documentation (cherry picked from commit eef779b8d631de971d440051cae21040f4de558f) Signed-off-by: Xiangrui Meng --- docs/mllib-feature-extraction.md | 63 +++++++++++++++++++++++++++++++- 1 file changed, 62 insertions(+), 1 deletion(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 21453cb9cd8c9..4b3cb715c58c7 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -9,4 +9,65 @@ displayTitle: MLlib - Feature Extraction ## Word2Vec -## TFIDF +Word2Vec computes distributed vector representation of words. The main advantage of the distributed +representations is that similar words are close in the vector space, which makes generalization to +novel patterns easier and model estimation more robust. Distributed vector representation is +showed to be useful in many natural language processing applications such as named entity +recognition, disambiguation, parsing, tagging and machine translation. + +### Model + +In our implementation of Word2Vec, we used skip-gram model. The training objective of skip-gram is +to learn word vector representations that are good at predicting its context in the same sentence. +Mathematically, given a sequence of training words `$w_1, w_2, \dots, w_T$`, the objective of the +skip-gram model is to maximize the average log-likelihood +`\[ +\frac{1}{T} \sum_{t = 1}^{T}\sum_{j=-k}^{j=k} \log p(w_{t+j} | w_t) +\]` +where $k$ is the size of the training window. + +In the skip-gram model, every word $w$ is associated with two vectors $u_w$ and $v_w$ which are +vector representations of $w$ as word and context respectively. The probability of correctly +predicting word $w_i$ given word $w_j$ is determined by the softmax model, which is +`\[ +p(w_i | w_j ) = \frac{\exp(u_{w_i}^{\top}v_{w_j})}{\sum_{l=1}^{V} \exp(u_l^{\top}v_{w_j})} +\]` +where $V$ is the vocabulary size. + +The skip-gram model with softmax is expensive because the cost of computing $\log p(w_i | w_j)$ +is proportional to $V$, which can be easily in order of millions. To speed up training of Word2Vec, +we used hierarchical softmax, which reduced the complexity of computing of $\log p(w_i | w_j)$ to +$O(\log(V))$ + +### Example + +The example below demonstrates how to load a text file, parse it as an RDD of `Seq[String]`, +construct a `Word2Vec` instance and then fit a `Word2VecModel` with the input data. Finally, +we display the top 40 synonyms of the specified word. To run the example, first download +the [text8](http://mattmahoney.net/dc/text8.zip) data and extract it to your preferred directory. +Here we assume the extracted file is `text8` and in same directory as you run the spark shell. + +
    +
    +{% highlight scala %} +import org.apache.spark._ +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Word2Vec + +val input = sc.textFile("text8").map(line => line.split(" ").toSeq) + +val word2vec = new Word2Vec() + +val model = word2vec.fit(input) + +val synonyms = model.findSynonyms("china", 40) + +for((synonym, cosineSimilarity) <- synonyms) { + println(s"$synonym $cosineSimilarity") +} +{% endhighlight %} +
    +
    + +## TFIDF \ No newline at end of file From e0bc333b6ad36feac5397600fe6948dcb37a8e44 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 18 Aug 2014 01:15:45 -0700 Subject: [PATCH 025/192] [MLlib] Remove transform(dataset: RDD[String]) from Word2Vec public API mengxr Remove transform(dataset: RDD[String]) from public API. Author: Liquan Pei Closes #2010 from Ishiihara/Word2Vec-api and squashes the following commits: 17b1031 [Liquan Pei] remove transform(dataset: RDD[String]) from public API (cherry picked from commit 9306b8c6c8c412b9d0d5cffb6bd7a87784f0f6bf) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index d2ae62b482aff..1dcaa2cd2e630 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -434,15 +434,6 @@ class Word2VecModel private[mllib] ( } } - /** - * Transforms an RDD to its vector representation - * @param dataset a an RDD of words - * @return RDD of vector representation - */ - def transform(dataset: RDD[String]): RDD[Vector] = { - dataset.map(word => transform(word)) - } - /** * Find synonyms of a word * @param word a word From 12f16ba3fa1f3cde9f43c094029017f4192b1bac Mon Sep 17 00:00:00 2001 From: Chandan Kumar Date: Mon, 18 Aug 2014 09:52:25 -0700 Subject: [PATCH 026/192] [SPARK-2862] histogram method fails on some choices of bucketCount Author: Chandan Kumar Closes #1787 from nrchandan/spark-2862 and squashes the following commits: a76bbf6 [Chandan Kumar] [SPARK-2862] Fix for a broken test case and add new test cases 4211eea [Chandan Kumar] [SPARK-2862] Add Scala bug id 13854f1 [Chandan Kumar] [SPARK-2862] Use shorthand range notation to avoid Scala bug (cherry picked from commit f45efbb8aaa65bc46d65e77e93076fbc29f4455d) Signed-off-by: Xiangrui Meng --- .../apache/spark/rdd/DoubleRDDFunctions.scala | 15 ++++++++---- .../org/apache/spark/rdd/DoubleRDDSuite.scala | 23 +++++++++++++++++++ 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index f233544d128f5..e0494ee39657c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -95,7 +95,12 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * If the elements in RDD do not vary (max == min) always returns a single bucket. */ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { - // Compute the minimum and the maxium + // Scala's built-in range has issues. See #SI-8782 + def customRange(min: Double, max: Double, steps: Int): IndexedSeq[Double] = { + val span = max - min + Range.Int(0, steps, 1).map(s => min + (s * span) / steps) :+ max + } + // Compute the minimum and the maximum val (max: Double, min: Double) = self.mapPartitions { items => Iterator(items.foldRight(Double.NegativeInfinity, Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) => @@ -107,9 +112,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { throw new UnsupportedOperationException( "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } - val increment = (max-min)/bucketCount.toDouble - val range = if (increment != 0) { - Range.Double.inclusive(min, max, increment) + val range = if (min != max) { + // Range.Double.inclusive(min, max, increment) + // The above code doesn't always work. See Scala bug #SI-8782. + // https://issues.scala-lang.org/browse/SI-8782 + customRange(min, max, bucketCount) } else { List(min, min) } diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index a822bd18bfdbd..f89bdb6e07dea 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -245,6 +245,29 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithoutBucketsForLargerDatasets") { + // Verify the case of slighly larger datasets + val rdd = sc.parallelize(6 to 99) + val (histogramBuckets, histogramResults) = rdd.histogram(8) + val expectedHistogramResults = + Array(12, 12, 11, 12, 12, 11, 12, 12) + val expectedHistogramBuckets = + Array(6.0, 17.625, 29.25, 40.875, 52.5, 64.125, 75.75, 87.375, 99.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + + test("WorksWithoutBucketsWithIrrationalBucketEdges") { + // Verify the case of buckets with irrational edges. See #SPARK-2862. + val rdd = sc.parallelize(6 to 99) + val (histogramBuckets, histogramResults) = rdd.histogram(9) + val expectedHistogramResults = + Array(11, 10, 11, 10, 10, 11, 10, 10, 11) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets(0) === 6.0) + assert(histogramBuckets(9) === 99.0) + } + // Test the failure mode with an invalid RDD test("ThrowsExceptionOnInvalidRDDs") { // infinity From ec0b91edd592cf89be349e0e5ad7553e02f70cd3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 18 Aug 2014 10:00:46 -0700 Subject: [PATCH 027/192] SPARK-3096: Include parquet hive serde by default in build A small change - we should just add this dependency. It doesn't have any recursive deps and it's needed for reading have parquet tables. Author: Patrick Wendell Closes #2009 from pwendell/parquet and squashes the following commits: e411f9f [Patrick Wendell] SPARk-309: Include parquet hive serde by default in build (cherry picked from commit 7ae28d1247e4756219016206c51fec1656e3917b) Signed-off-by: Michael Armbrust --- sql/hive/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c18a664e737c8..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -35,6 +35,11 @@ + + com.twitter + parquet-hive-bundle + 1.5.0 + org.apache.spark spark-core_${scala.binary.version} From 55e9dd637bdef3a2acf56af95410219e23c9502a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 10:05:52 -0700 Subject: [PATCH 028/192] [SPARK-3084] [SQL] Collect broadcasted tables in parallel in joins BroadcastHashJoin has a broadcastFuture variable that tries to collect the broadcasted table in a separate thread, but this doesn't help because it's a lazy val that only gets initialized when you attempt to build the RDD. Thus queries that broadcast multiple tables would collect and broadcast them sequentially. I changed this to a val to let it start collecting right when the operator is created. Author: Matei Zaharia Closes #1990 from mateiz/spark-3084 and squashes the following commits: f468766 [Matei Zaharia] [SPARK-3084] Collect broadcasted tables in parallel in joins (cherry picked from commit 6a13dca12fac06f3af892ffcc8922cc84f91b786) Signed-off-by: Michael Armbrust --- .../src/main/scala/org/apache/spark/sql/execution/joins.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index c86811e838bd8..481bb8c05e71b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -424,7 +424,7 @@ case class BroadcastHashJoin( UnspecifiedDistribution :: UnspecifiedDistribution :: Nil @transient - lazy val broadcastFuture = future { + val broadcastFuture = future { sparkContext.broadcast(buildPlan.executeCollect()) } From 4da76fc81c224b04bd652c4a72fb77516a32de0c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 10:45:24 -0700 Subject: [PATCH 029/192] [SPARK-3085] [SQL] Use compact data structures in SQL joins This reuses the CompactBuffer from Spark Core to save memory and pointer dereferences. I also tried AppendOnlyMap instead of java.util.HashMap but unfortunately that slows things down because it seems to do more equals() calls and the equals on GenericRow, and especially JoinedRow, is pretty expensive. Author: Matei Zaharia Closes #1993 from mateiz/spark-3085 and squashes the following commits: 188221e [Matei Zaharia] Remove unneeded import 5f903ee [Matei Zaharia] [SPARK-3085] [SQL] Use compact data structures in SQL joins (cherry picked from commit 4bf3de71074053af94f077c99e9c65a1962739e1) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/execution/joins.scala | 67 +++++++++---------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 481bb8c05e71b..b08f9aacc1fcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -19,16 +19,15 @@ package org.apache.spark.sql.execution import java.util.{HashMap => JavaHashMap} -import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ import scala.concurrent.duration._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.util.collection.CompactBuffer @DeveloperApi sealed abstract class BuildSide @@ -67,7 +66,7 @@ trait HashJoin { def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { // TODO: Use Spark's HashMap implementation. - val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() + val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() var currentRow: Row = null // Create a mapping of buildKeys -> rows @@ -77,7 +76,7 @@ trait HashJoin { if (!rowKey.anyNull) { val existingMatchList = hashTable.get(rowKey) val matchList = if (existingMatchList == null) { - val newMatchList = new ArrayBuffer[Row]() + val newMatchList = new CompactBuffer[Row]() hashTable.put(rowKey, newMatchList) newMatchList } else { @@ -89,7 +88,7 @@ trait HashJoin { new Iterator[Row] { private[this] var currentStreamedRow: Row = _ - private[this] var currentHashMatches: ArrayBuffer[Row] = _ + private[this] var currentHashMatches: CompactBuffer[Row] = _ private[this] var currentMatchPosition: Int = -1 // Mutable per row objects. @@ -140,7 +139,7 @@ trait HashJoin { /** * :: DeveloperApi :: - * Performs a hash based outer join for two child relations by shuffling the data using + * Performs a hash based outer join for two child relations by shuffling the data using * the join keys. This operator requires loading the associated partition in both side into memory. */ @DeveloperApi @@ -179,26 +178,26 @@ case class HashOuterJoin( @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala - // iterator for performance purpose. + // iterator for performance purpose. private[this] def leftOuterIterator( key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { val joinedRow = new JoinedRow() val rightNullRow = new GenericRow(right.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - leftIter.iterator.flatMap { l => + leftIter.iterator.flatMap { l => joinedRow.withLeft(l) var matched = false - (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => + (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => matched = true joinedRow.copy } else { Nil }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the + // as we don't know whether we need to append it until finish iterating all of the // records in right side. // If we didn't get any proper row, then append a single row with empty right joinedRow.withRight(rightNullRow).copy @@ -210,20 +209,20 @@ case class HashOuterJoin( key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { val joinedRow = new JoinedRow() val leftNullRow = new GenericRow(left.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - rightIter.iterator.flatMap { r => + rightIter.iterator.flatMap { r => joinedRow.withRight(r) var matched = false - (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => + (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => matched = true joinedRow.copy } else { Nil }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the + // as we don't know whether we need to append it until finish iterating all of the // records in left side. // If we didn't get any proper row, then append a single row with empty left. joinedRow.withLeft(leftNullRow).copy @@ -236,7 +235,7 @@ case class HashOuterJoin( val joinedRow = new JoinedRow() val leftNullRow = new GenericRow(left.output.length) val rightNullRow = new GenericRow(right.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) if (!key.anyNull) { @@ -246,8 +245,8 @@ case class HashOuterJoin( leftIter.iterator.flatMap[Row] { l => joinedRow.withLeft(l) var matched = false - rightIter.zipWithIndex.collect { - // 1. For those matched (satisfy the join condition) records with both sides filled, + rightIter.zipWithIndex.collect { + // 1. For those matched (satisfy the join condition) records with both sides filled, // append them directly case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { @@ -260,7 +259,7 @@ case class HashOuterJoin( // 2. For those unmatched records in left, append additional records with empty right. // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all + // as we don't know whether we need to append it until finish iterating all // of the records in right side. // If we didn't get any proper row, then append a single row with empty right. joinedRow.withRight(rightNullRow).copy @@ -268,8 +267,8 @@ case class HashOuterJoin( } ++ rightIter.zipWithIndex.collect { // 3. For those unmatched records in right, append additional records with empty left. - // Re-visiting the records in right, and append additional row with empty left, if its not - // in the matched set. + // Re-visiting the records in right, and append additional row with empty left, if its not + // in the matched set. case (r, idx) if (!rightMatchedSet.contains(idx)) => { joinedRow(leftNullRow, r).copy } @@ -284,15 +283,15 @@ case class HashOuterJoin( } private[this] def buildHashTable( - iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, ArrayBuffer[Row]] = { - val hashTable = new JavaHashMap[Row, ArrayBuffer[Row]]() + iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, CompactBuffer[Row]] = { + val hashTable = new JavaHashMap[Row, CompactBuffer[Row]]() while (iter.hasNext) { val currentRow = iter.next() val rowKey = keyGenerator(currentRow) var existingMatchList = hashTable.get(rowKey) if (existingMatchList == null) { - existingMatchList = new ArrayBuffer[Row]() + existingMatchList = new CompactBuffer[Row]() hashTable.put(rowKey, existingMatchList) } @@ -311,20 +310,20 @@ case class HashOuterJoin( val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) import scala.collection.JavaConversions._ - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) joinType match { case LeftOuter => leftHashTable.keysIterator.flatMap { key => - leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case RightOuter => rightHashTable.keysIterator.flatMap { key => - rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => - fullOuterIterator(key, - leftHashTable.getOrElse(key, EMPTY_LIST), + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") @@ -550,7 +549,7 @@ case class BroadcastNestedLoopJoin( /** All rows that either match both-way, or rows from streamed joined with nulls. */ val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => - val matchedRows = new ArrayBuffer[Row] + val matchedRows = new CompactBuffer[Row] // TODO: Use Spark's BitSet. val includedBroadcastTuples = new scala.collection.mutable.BitSet(broadcastedRelation.value.size) @@ -602,20 +601,20 @@ case class BroadcastNestedLoopJoin( val rightNulls = new GenericMutableRow(right.output.size) /** Rows from broadcasted joined with nulls. */ val broadcastRowsWithNulls: Seq[Row] = { - val arrBuf: collection.mutable.ArrayBuffer[Row] = collection.mutable.ArrayBuffer() + val buf: CompactBuffer[Row] = new CompactBuffer() var i = 0 val rel = broadcastedRelation.value while (i < rel.length) { if (!allIncludedBroadcastTuples.contains(i)) { (joinType, buildSide) match { - case (RightOuter | FullOuter, BuildRight) => arrBuf += new JoinedRow(leftNulls, rel(i)) - case (LeftOuter | FullOuter, BuildLeft) => arrBuf += new JoinedRow(rel(i), rightNulls) + case (RightOuter | FullOuter, BuildRight) => buf += new JoinedRow(leftNulls, rel(i)) + case (LeftOuter | FullOuter, BuildLeft) => buf += new JoinedRow(rel(i), rightNulls) case _ => } } i += 1 } - arrBuf.toSeq + buf.toSeq } // TODO: Breaks lineage. From 496f62d9a98067256d8a51fd1e7a485ff6492fa8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 18 Aug 2014 10:52:20 -0700 Subject: [PATCH 030/192] SPARK-3025 [SQL]: Allow JDBC clients to set a fair scheduler pool This definitely needs review as I am not familiar with this part of Spark. I tested this locally and it did seem to work. Author: Patrick Wendell Closes #1937 from pwendell/scheduler and squashes the following commits: b858e33 [Patrick Wendell] SPARK-3025: Allow JDBC clients to set a fair scheduler pool (cherry picked from commit 6bca8898a1aa4ca7161492229bac1748b3da2ad7) Signed-off-by: Michael Armbrust --- docs/sql-programming-guide.md | 5 ++++ .../scala/org/apache/spark/sql/SQLConf.scala | 3 +++ .../server/SparkSQLOperationManager.scala | 27 ++++++++++++++----- 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index cd6543945c385..34accade36ea9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -605,6 +605,11 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script comes with Hive. +To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, +users can set the `spark.sql.thriftserver.scheduler.pool` variable: + + SET spark.sql.thriftserver.scheduler.pool=accounting; + ### Migration Guide for Shark Users #### Reducer number diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 90de11182e605..56face2992bcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -33,6 +33,9 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + // This is only used for the thriftserver + val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 9338e8121b0fe..699a1103f3248 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -17,24 +17,24 @@ package org.apache.spark.sql.hive.thriftserver.server -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.math.{random, round} - import java.sql.Timestamp import java.util.{Map => JMap} +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map} +import scala.math.{random, round} + import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession - import org.apache.spark.Logging +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} +import org.apache.spark.sql.catalyst.plans.logical.SetCommand import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -43,6 +43,9 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + // TODO: Currenlty this will grow infinitely, even as sessions expire + val sessionToActivePool = Map[HiveSession, String]() + override def newExecuteStatementOperation( parentSession: HiveSession, statement: String, @@ -165,8 +168,18 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage try { result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) + result.queryExecution.logical match { + case SetCommand(Some(key), Some(value)) if (key == SQLConf.THRIFTSERVER_POOL) => + sessionToActivePool(parentSession) = value + logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + case _ => + } + val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) + sessionToActivePool.get(parentSession).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } iter = { val resultRdd = result.queryExecution.toRdd val useIncrementalCollect = From 2ae2857986e94d5a8bd5f4660eabe5689463bd21 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 11:00:10 -0700 Subject: [PATCH 031/192] [SPARK-3091] [SQL] Add support for caching metadata on Parquet files For larger Parquet files, reading the file footers (which is done in parallel on up to 5 threads) and HDFS block locations (which is serial) can take multiple seconds. We can add an option to cache this data within FilteringParquetInputFormat. Unfortunately ParquetInputFormat only caches footers within each instance of ParquetInputFormat, not across them. Note: this PR leaves this turned off by default for 1.1, but I believe it's safe to turn it on after. The keys in the hash maps are FileStatus objects that include a modification time, so this will work fine if files are modified. The location cache could become invalid if files have moved within HDFS, but that's rare so I just made it invalidate entries every 15 minutes. Author: Matei Zaharia Closes #2005 from mateiz/parquet-cache and squashes the following commits: dae8efe [Matei Zaharia] Bug fix c71e9ed [Matei Zaharia] Handle empty statuses directly 22072b0 [Matei Zaharia] Use Guava caches and add a config option for caching metadata 8fb56ce [Matei Zaharia] Cache file block locations too 453bd21 [Matei Zaharia] Bug fix 4094df6 [Matei Zaharia] First attempt at caching Parquet footers (cherry picked from commit 9eb74c7d2cbe127dd4c32bf1a8318497b2fb55b6) Signed-off-by: Michael Armbrust --- .../scala/org/apache/spark/sql/SQLConf.scala | 1 + .../sql/parquet/ParquetTableOperations.scala | 84 ++++++++++++++++--- 2 files changed, 72 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 56face2992bcf..4f2adb006fbc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -32,6 +32,7 @@ private[spark] object SQLConf { val CODEGEN_ENABLED = "spark.sql.codegen" val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 759a2a586b926..c6dca10f6ad7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -17,22 +17,23 @@ package org.apache.spark.sql.parquet -import scala.collection.JavaConversions._ -import scala.collection.mutable -import scala.util.Try - import java.io.IOException import java.lang.{Long => JLong} import java.text.SimpleDateFormat -import java.util.{Date, List => JList} +import java.util.concurrent.{Callable, TimeUnit} +import java.util.{ArrayList, Collections, Date, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.util.Try + +import com.google.common.cache.CacheBuilder import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData @@ -41,7 +42,7 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} import org.apache.spark.{Logging, SerializableWritable, TaskContext} @@ -96,6 +97,11 @@ case class ParquetTableScan( ParquetFilters.serializeFilterExpressions(columnPruningPred, conf) } + // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + conf.set( + SQLConf.PARQUET_CACHE_METADATA, + sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "false")) + sc.newAPIHadoopRDD( conf, classOf[FilteringParquetRowInputFormat], @@ -323,10 +329,40 @@ private[parquet] class FilteringParquetRowInputFormat } override def getFooters(jobContext: JobContext): JList[Footer] = { + import FilteringParquetRowInputFormat.footerCache + if (footers eq null) { + val conf = ContextUtil.getConfiguration(jobContext) + val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) val statuses = listStatus(jobContext) fileStatuses = statuses.map(file => file.getPath -> file).toMap - footers = getFooters(ContextUtil.getConfiguration(jobContext), statuses) + if (statuses.isEmpty) { + footers = Collections.emptyList[Footer] + } else if (!cacheMetadata) { + // Read the footers from HDFS + footers = getFooters(conf, statuses) + } else { + // Read only the footers that are not in the footerCache + val foundFooters = footerCache.getAllPresent(statuses) + val toFetch = new ArrayList[FileStatus] + for (s <- statuses) { + if (!foundFooters.containsKey(s)) { + toFetch.add(s) + } + } + val newFooters = new mutable.HashMap[FileStatus, Footer] + if (toFetch.size > 0) { + val fetched = getFooters(conf, toFetch) + for ((status, i) <- toFetch.zipWithIndex) { + newFooters(status) = fetched.get(i) + } + footerCache.putAll(newFooters) + } + footers = new ArrayList[Footer](statuses.size) + for (status <- statuses) { + footers.add(newFooters.getOrElse(status, foundFooters.get(status))) + } + } } footers @@ -339,6 +375,10 @@ private[parquet] class FilteringParquetRowInputFormat configuration: Configuration, footers: JList[Footer]): JList[ParquetInputSplit] = { + import FilteringParquetRowInputFormat.blockLocationCache + + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) val minSplitSize: JLong = Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) @@ -366,16 +406,23 @@ private[parquet] class FilteringParquetRowInputFormat for (footer <- footers) { val fs = footer.getFile.getFileSystem(configuration) val file = footer.getFile - val fileStatus = fileStatuses.getOrElse(file, fs.getFileStatus(file)) + val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) val parquetMetaData = footer.getParquetMetadata val blocks = parquetMetaData.getBlocks - val fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen) + var blockLocations: Array[BlockLocation] = null + if (!cacheMetadata) { + blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) + } else { + blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { + def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) + }) + } splits.addAll( generateSplits.invoke( null, blocks, - fileBlockLocations, - fileStatus, + blockLocations, + status, parquetMetaData.getFileMetaData, readContext.getRequestedSchema.toString, readContext.getReadSupportMetadata, @@ -387,6 +434,17 @@ private[parquet] class FilteringParquetRowInputFormat } } +private[parquet] object FilteringParquetRowInputFormat { + private val footerCache = CacheBuilder.newBuilder() + .maximumSize(20000) + .build[FileStatus, Footer]() + + private val blockLocationCache = CacheBuilder.newBuilder() + .maximumSize(20000) + .expireAfterWrite(15, TimeUnit.MINUTES) // Expire locations since HDFS files might move + .build[FileStatus, Array[BlockLocation]]() +} + private[parquet] object FileSystemHelper { def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) From cc4015d2fa3785b92e6ab079b3abcf17627f7c56 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Aug 2014 13:17:10 -0700 Subject: [PATCH 032/192] [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables. This PR adds an experimental flag `spark.sql.hive.convertMetastoreParquet` that when true causes the planner to detects tables that use Hive's Parquet SerDe and instead plans them using Spark SQL's native `ParquetTableScan`. Author: Michael Armbrust Author: Yin Huai Closes #1819 from marmbrus/parquetMetastore and squashes the following commits: 1620079 [Michael Armbrust] Revert "remove hive parquet bundle" cc30430 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore 4f3d54f [Michael Armbrust] fix style 41ebc5f [Michael Armbrust] remove hive parquet bundle a43e0da [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore 4c4dc19 [Michael Armbrust] Fix bug with tree splicing. ebb267e [Michael Armbrust] include parquet hive to tests pass (Remove this later). c0d9b72 [Michael Armbrust] Avoid creating a HadoopRDD per partition. Add dirty hacks to retrieve partition values from the InputSplit. 8cdc93c [Michael Armbrust] Merge pull request #8 from yhuai/parquetMetastore a0baec7 [Yin Huai] Partitioning columns can be resolved. 1161338 [Michael Armbrust] Add a test to make sure conversion is actually happening 212d5cd [Michael Armbrust] Initial support for using ParquetTableScan to read HiveMetaStore tables. (cherry picked from commit 3abd0c1cda09bb575adc99847a619bc84af37fd0) Signed-off-by: Michael Armbrust --- project/SparkBuild.scala | 1 - .../spark/sql/execution/basicOperators.scala | 12 ++ .../spark/sql/parquet/ParquetRelation.scala | 8 +- .../sql/parquet/ParquetTableOperations.scala | 74 ++++++-- .../apache/spark/sql/hive/HiveContext.scala | 9 + .../spark/sql/hive/HiveStrategies.scala | 119 +++++++++++- .../sql/hive/parquet/FakeParquetSerDe.scala | 56 ++++++ .../sql/parquet/ParquetMetastoreSuite.scala | 171 ++++++++++++++++++ 8 files changed, 427 insertions(+), 23 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 63a285b81a60c..49d52aefca17a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -228,7 +228,6 @@ object SQL { object Hive { lazy val settings = Seq( - javaOptions += "-XX:MaxPermSize=1g", // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 0027f3cf1fc79..f9dfa3c92f1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -303,3 +303,15 @@ case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { left.execute().map(_.copy()).intersection(right.execute().map(_.copy())) } } + +/** + * :: DeveloperApi :: + * A plan node that does nothing but lie about the output of its child. Used to spice a + * (hopefully structurally equivalent) tree from a different optimization sequence into an already + * resolved tree. + */ +@DeveloperApi +case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { + def children = child :: Nil + def execute() = child.execute() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 053b2a154389c..1713ae6fb5d93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -47,7 +47,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} private[sql] case class ParquetRelation( path: String, @transient conf: Option[Configuration], - @transient sqlContext: SQLContext) + @transient sqlContext: SQLContext, + partitioningAttributes: Seq[Attribute] = Nil) extends LeafNode with MultiInstanceRelation { self: Product => @@ -61,12 +62,13 @@ private[sql] case class ParquetRelation( /** Attributes */ override val output = + partitioningAttributes ++ ParquetTypesConverter.readSchemaFromFile( - new Path(path), + new Path(path.split(",").head), conf, sqlContext.isParquetBinaryAsString) - override def newInstance = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] + override def newInstance() = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index c6dca10f6ad7c..f6cfab736d98a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData @@ -42,6 +43,7 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} @@ -60,11 +62,18 @@ case class ParquetTableScan( // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes // by exprId. note: output cannot be transient, see // https://issues.apache.org/jira/browse/SPARK-1367 - val output = attributes.map { a => - relation.output - .find(o => o.exprId == a.exprId) - .getOrElse(sys.error(s"Invalid parquet attribute $a in ${relation.output.mkString(",")}")) - } + val normalOutput = + attributes + .filterNot(a => relation.partitioningAttributes.map(_.exprId).contains(a.exprId)) + .flatMap(a => relation.output.find(o => o.exprId == a.exprId)) + + val partOutput = + attributes.flatMap(a => relation.partitioningAttributes.find(o => o.exprId == a.exprId)) + + def output = partOutput ++ normalOutput + + assert(normalOutput.size + partOutput.size == attributes.size, + s"$normalOutput + $partOutput != $attributes, ${relation.output}") override def execute(): RDD[Row] = { val sc = sqlContext.sparkContext @@ -72,16 +81,19 @@ case class ParquetTableScan( ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) - val qualifiedPath = { - val path = new Path(relation.path) - path.getFileSystem(conf).makeQualified(path) + + relation.path.split(",").foreach { curPath => + val qualifiedPath = { + val path = new Path(curPath) + path.getFileSystem(conf).makeQualified(path) + } + NewFileInputFormat.addInputPath(job, qualifiedPath) } - NewFileInputFormat.addInputPath(job, qualifiedPath) // Store both requested and original schema in `Configuration` conf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(output)) + ParquetTypesConverter.convertToString(normalOutput)) conf.set( RowWriteSupport.SPARK_ROW_SCHEMA, ParquetTypesConverter.convertToString(relation.output)) @@ -102,13 +114,41 @@ case class ParquetTableScan( SQLConf.PARQUET_CACHE_METADATA, sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "false")) - sc.newAPIHadoopRDD( - conf, - classOf[FilteringParquetRowInputFormat], - classOf[Void], - classOf[Row]) - .map(_._2) - .filter(_ != null) // Parquet's record filters may produce null values + val baseRDD = + new org.apache.spark.rdd.NewHadoopRDD( + sc, + classOf[FilteringParquetRowInputFormat], + classOf[Void], + classOf[Row], + conf) + + if (partOutput.nonEmpty) { + baseRDD.mapPartitionsWithInputSplit { case (split, iter) => + val partValue = "([^=]+)=([^=]+)".r + val partValues = + split.asInstanceOf[parquet.hadoop.ParquetInputSplit] + .getPath + .toString + .split("/") + .flatMap { + case partValue(key, value) => Some(key -> value) + case _ => None + }.toMap + + val partitionRowValues = + partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) + + new Iterator[Row] { + private[this] val joinedRow = new JoinedRow(Row(partitionRowValues:_*), null) + + def hasNext = iter.hasNext + + def next() = joinedRow.withRight(iter.next()._2) + } + } + } else { + baseRDD.map(_._2) + }.filter(_ != null) // Parquet's record filters may produce null values } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index a8da676ffa0e0..ff32c7c90a0d2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -79,6 +79,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // Change the default SQL dialect to HiveQL override private[spark] def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + /** + * When true, enables an experimental feature where metastore tables that use the parquet SerDe + * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive + * SerDe. + */ + private[spark] def convertMetastoreParquet: Boolean = + getConf("spark.sql.hive.convertMetastoreParquet", "false") == "true" + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } @@ -326,6 +334,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { TakeOrdered, ParquetOperations, InMemoryScans, + ParquetConversion, // Must be before HiveTableScans HiveTableScans, DataSinks, Scripts, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5fcc1bd4b9adf..389ace726d205 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,14 +17,20 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.SQLContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.columnar.InMemoryRelation +import org.apache.spark.sql.parquet.{ParquetRelation, ParquetTableScan} + +import scala.collection.JavaConversions._ private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -32,6 +38,115 @@ private[hive] trait HiveStrategies { val hiveContext: HiveContext + /** + * :: Experimental :: + * Finds table scans that would use the Hive SerDe and replaces them with our own native parquet + * table scan operator. + * + * TODO: Much of this logic is duplicated in HiveTableScan. Ideally we would do some refactoring + * but since this is after the code freeze for 1.1 all logic is here to minimize disruption. + * + * Other issues: + * - Much of this logic assumes case insensitive resolution. + */ + @Experimental + object ParquetConversion extends Strategy { + implicit class LogicalPlanHacks(s: SchemaRDD) { + def lowerCase = + new SchemaRDD(s.sqlContext, LowerCaseSchema(s.logicalPlan)) + + def addPartitioningAttributes(attrs: Seq[Attribute]) = + new SchemaRDD( + s.sqlContext, + s.logicalPlan transform { + case p: ParquetRelation => p.copy(partitioningAttributes = attrs) + }) + } + + implicit class PhysicalPlanHacks(originalPlan: SparkPlan) { + def fakeOutput(newOutput: Seq[Attribute]) = + OutputFaker( + originalPlan.output.map(a => + newOutput.find(a.name.toLowerCase == _.name.toLowerCase) + .getOrElse( + sys.error(s"Can't find attribute $a to fake in set ${newOutput.mkString(",")}"))), + originalPlan) + } + + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) + if relation.tableDesc.getSerdeClassName.contains("Parquet") && + hiveContext.convertMetastoreParquet => + + // Filter out all predicates that only deal with partition keys + val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val (pruningPredicates, otherPredicates) = predicates.partition { + _.references.map(_.exprId).subsetOf(partitionKeyIds) + } + + // We are going to throw the predicates and projection back at the whole optimization + // sequence so lets unresolve all the attributes, allowing them to be rebound to the + // matching parquet attributes. + val unresolvedOtherPredicates = otherPredicates.map(_ transform { + case a: AttributeReference => UnresolvedAttribute(a.name) + }).reduceOption(And).getOrElse(Literal(true)) + + val unresolvedProjection = projectList.map(_ transform { + case a: AttributeReference => UnresolvedAttribute(a.name) + }) + + if (relation.hiveQlTable.isPartitioned) { + val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) + // Translate the predicate so that it automatically casts the input values to the correct + // data types during evaluation + val castedPredicate = rawPredicate transform { + case a: AttributeReference => + val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) + val key = relation.partitionKeys(idx) + Cast(BoundReference(idx, StringType, nullable = true), key.dataType) + } + + val inputData = new GenericMutableRow(relation.partitionKeys.size) + val pruningCondition = + if(codegenEnabled) { + GeneratePredicate(castedPredicate) + } else { + InterpretedPredicate(castedPredicate) + } + + val partitions = relation.hiveQlPartitions.filter { part => + val partitionValues = part.getValues + var i = 0 + while (i < partitionValues.size()) { + inputData(i) = partitionValues(i) + i += 1 + } + pruningCondition(inputData) + } + + hiveContext + .parquetFile(partitions.map(_.getLocation).mkString(",")) + .addPartitioningAttributes(relation.partitionKeys) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection:_*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)):: Nil + } else { + hiveContext + .parquetFile(relation.hiveQlTable.getDataLocation.getPath) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection:_*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } + case _ => Nil + } + } + object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala new file mode 100644 index 0000000000000..544abfc32423c --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -0,0 +1,56 @@ +/* + * 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.sql.hive.parquet + +import java.util.Properties + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category +import org.apache.hadoop.hive.serde2.{SerDeStats, SerDe} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.io.Writable + +/** + * A placeholder that allows SparkSQL users to create metastore tables that are stored as + * parquet files. It is only intended to pass the checks that the serde is valid and exists + * when a CREATE TABLE is run. The actual work of decoding will be done by ParquetTableScan + * when "spark.sql.hive.convertMetastoreParquet" is set to true. + */ +@deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " + + "placeholder in the Hive MetaStore") +class FakeParquetSerDe extends SerDe { + override def getObjectInspector: ObjectInspector = new ObjectInspector { + override def getCategory: Category = Category.PRIMITIVE + + override def getTypeName: String = "string" + } + + override def deserialize(p1: Writable): AnyRef = throwError + + override def initialize(p1: Configuration, p2: Properties): Unit = {} + + override def getSerializedClass: Class[_ <: Writable] = throwError + + override def getSerDeStats: SerDeStats = throwError + + override def serialize(p1: scala.Any, p2: ObjectInspector): Writable = throwError + + private def throwError = + sys.error( + "spark.sql.hive.convertMetastoreParquet must be set to true to use FakeParquetSerDe") +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala new file mode 100644 index 0000000000000..0723be7298e15 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -0,0 +1,171 @@ + +/* + * 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.sql.parquet + +import java.io.File + +import org.apache.spark.sql.hive.execution.HiveTableScan +import org.scalatest.BeforeAndAfterAll + +import scala.reflect.ClassTag + +import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +case class ParquetData(intField: Int, stringField: String) + +/** + * Tests for our SerDe -> Native parquet scan conversion. + */ +class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "true") + } + + override def afterAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "false") + } + + val partitionedTableDir = File.createTempFile("parquettests", "sparksql") + partitionedTableDir.delete() + partitionedTableDir.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDir, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + + sql(s""" + create external table partitioned_parquet + ( + intField INT, + stringField STRING + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${partitionedTableDir.getCanonicalPath}' + """) + + sql(s""" + create external table normal_parquet + ( + intField INT, + stringField STRING + ) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + """) + + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + } + + test("project the partitioning column") { + checkAnswer( + sql("SELECT p, count(*) FROM partitioned_parquet group by p"), + (1, 10) :: + (2, 10) :: + (3, 10) :: + (4, 10) :: + (5, 10) :: + (6, 10) :: + (7, 10) :: + (8, 10) :: + (9, 10) :: + (10, 10) :: Nil + ) + } + + test("project partitioning and non-partitioning columns") { + checkAnswer( + sql("SELECT stringField, p, count(intField) " + + "FROM partitioned_parquet GROUP BY p, stringField"), + ("part-1", 1, 10) :: + ("part-2", 2, 10) :: + ("part-3", 3, 10) :: + ("part-4", 4, 10) :: + ("part-5", 5, 10) :: + ("part-6", 6, 10) :: + ("part-7", 7, 10) :: + ("part-8", 8, 10) :: + ("part-9", 9, 10) :: + ("part-10", 10, 10) :: Nil + ) + } + + test("simple count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet"), + 100) + } + + test("pruned count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p = 1"), + 10) + } + + test("multi-partition pruned count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p IN (1,2,3)"), + 30) + } + + test("non-partition predicates") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE intField IN (1,2,3)"), + 30) + } + + test("sum") { + checkAnswer( + sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"), + 1 + 2 + 3 + ) + } + + test("non-part select(*)") { + checkAnswer( + sql("SELECT COUNT(*) FROM normal_parquet"), + 10 + ) + } + + test("conversion is working") { + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: HiveTableScan => true + }.isEmpty) + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: ParquetTableScan => true + }.nonEmpty) + } +} From e083334634ca0d7a25dee864fb2b9558ee92a2f7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 18 Aug 2014 13:58:35 -0700 Subject: [PATCH 033/192] [SPARK-3103] [PySpark] fix saveAsTextFile() with utf-8 bugfix: It will raise an exception when it try to encode non-ASCII strings into unicode. It should only encode unicode as "utf-8". Author: Davies Liu Closes #2018 from davies/fix_utf8 and squashes the following commits: 4db7967 [Davies Liu] fix saveAsTextFile() with utf-8 (cherry picked from commit d1d0ee41c27f1d07fed0c5d56ba26c723cc3dc26) Signed-off-by: Josh Rosen --- python/pyspark/rdd.py | 4 +++- python/pyspark/tests.py | 9 +++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 240381e5bae12..c708b69cc1e31 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1191,7 +1191,9 @@ def func(split, iterator): for x in iterator: if not isinstance(x, basestring): x = unicode(x) - yield x.encode("utf-8") + if isinstance(x, unicode): + x = x.encode("utf-8") + yield x keyed = self.mapPartitionsWithIndex(func) keyed._bypass_serializer = True keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f1fece998cd54..69d543d9d045d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -256,6 +256,15 @@ def test_save_as_textfile_with_unicode(self): raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + def test_save_as_textfile_with_utf8(self): + x = u"\u00A1Hola, mundo!" + data = self.sc.parallelize([x.encode("utf-8")]) + tempFile = tempfile.NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsTextFile(tempFile.name) + raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) + self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) From 25cabd7eec6e499fce94bce0d45087e9d8726a50 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 14:10:10 -0700 Subject: [PATCH 034/192] [SPARK-2718] [yarn] Handle quotes and other characters in user args. Due to the way Yarn runs things through bash, normal quoting doesn't work as expected. This change applies the necessary voodoo to the user args to avoid issues with bash and special characters. The change also uncovered an issue with the event logger app name sanitizing code; it wasn't cleaning up all "bad" characters, so sometimes it would fail to create the log dirs. I just added some more bad character replacements. Author: Marcelo Vanzin Closes #1724 from vanzin/SPARK-2718 and squashes the following commits: cc84b89 [Marcelo Vanzin] Review feedback. c1a257a [Marcelo Vanzin] Add test for backslashes. 55571d4 [Marcelo Vanzin] Unbreak yarn-client. 515613d [Marcelo Vanzin] [SPARK-2718] [yarn] Handle quotes and other characters in user args. (cherry picked from commit 6201b27643023569e19b68aa9d5c4e4e59ce0d79) Signed-off-by: Andrew Or --- .../scheduler/EventLoggingListener.scala | 3 +- .../yarn/ApplicationMasterArguments.scala | 6 +- .../apache/spark/deploy/yarn/ClientBase.scala | 9 +-- .../deploy/yarn/ExecutorRunnableUtil.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 25 ++++++++ .../yarn/YarnSparkHadoopUtilSuite.scala | 64 +++++++++++++++++++ 6 files changed, 101 insertions(+), 10 deletions(-) create mode 100644 yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala 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 7378ce923f0ae..370fcd85aa680 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -54,7 +54,8 @@ private[spark] class EventLoggingListener( private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis + private val name = appName.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") + .toLowerCase + "-" + System.currentTimeMillis val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 4c383ab574abe..424b0fb0936f2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -29,7 +29,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) - + private def parseArgs(inputArgs: List[String]): Unit = { val userArgsBuffer = new ArrayBuffer[String]() @@ -47,7 +47,7 @@ class ApplicationMasterArguments(val args: Array[String]) { userClass = value args = tail - case ("--args") :: value :: tail => + case ("--args" | "--arg") :: value :: tail => userArgsBuffer += value args = tail @@ -75,7 +75,7 @@ class ApplicationMasterArguments(val args: Array[String]) { userArgs = userArgsBuffer.readOnly } - + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { if (unknownParam != null) { System.err.println("Unknown/unsupported param " + unknownParam) 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 1da0a1b675554..3897b3a373a8c 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 @@ -300,11 +300,11 @@ trait ClientBase extends Logging { } def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --args " + val prefix = " --arg " val args = clientArgs.userArgs val retval = new StringBuilder() for (arg <- args) { - retval.append(prefix).append(" '").append(arg).append("' ") + retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) } retval.toString } @@ -386,7 +386,7 @@ trait ClientBase extends Logging { // 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. for ((k, v) <- sparkConf.getAll) { - javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" + javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } if (args.amClass == classOf[ApplicationMaster].getName) { @@ -400,7 +400,8 @@ trait ClientBase extends Logging { // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ javaOpts ++ - Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + Seq(args.amClass, "--class", YarnSparkHadoopUtil.escapeForShell(args.userClass), + "--jar ", YarnSparkHadoopUtil.escapeForShell(args.userJar), userArgsToString(args), "--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, 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 71a9e42846b2b..312d82a649792 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 @@ -68,10 +68,10 @@ trait ExecutorRunnableUtil extends Logging { // authentication settings. sparkConf.getAll. filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. - foreach { case (k, v) => javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" } + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } sparkConf.getAkkaConf. - foreach { case (k, v) => javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" } + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e98308cdbd74e..10aef5eb2486f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -148,4 +148,29 @@ object YarnSparkHadoopUtil { } } + /** + * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands + * using `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. The + * argument is enclosed in single quotes and some key characters are escaped. + * + * @param arg A single argument. + * @return Argument quoted for execution via Yarn's generated shell script. + */ + def escapeForShell(arg: String): String = { + if (arg != null) { + val escaped = new StringBuilder("'") + for (i <- 0 to arg.length() - 1) { + arg.charAt(i) match { + case '$' => escaped.append("\\$") + case '"' => escaped.append("\\\"") + case '\'' => escaped.append("'\\''") + case c => escaped.append(c) + } + } + escaped.append("'").toString() + } else { + arg + } + } + } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala new file mode 100644 index 0000000000000..7650bd4396c12 --- /dev/null +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.io.{File, IOException} + +import com.google.common.io.{ByteStreams, Files} +import org.scalatest.{FunSuite, Matchers} + +import org.apache.spark.Logging + +class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { + + val hasBash = + try { + val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor() + exitCode == 0 + } catch { + case e: IOException => + false + } + + if (!hasBash) { + logWarning("Cannot execute bash, skipping bash tests.") + } + + def bashTest(name: String)(fn: => Unit) = + if (hasBash) test(name)(fn) else ignore(name)(fn) + + bashTest("shell script escaping") { + val scriptFile = File.createTempFile("script.", ".sh") + val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6") + try { + val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ") + Files.write(("bash -c \"echo " + argLine + "\"").getBytes(), scriptFile) + scriptFile.setExecutable(true) + + val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath())) + val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim() + val err = new String(ByteStreams.toByteArray(proc.getErrorStream())) + val exitCode = proc.waitFor() + exitCode should be (0) + out should be (args.mkString(" ")) + } finally { + scriptFile.delete() + } + } + +} From 98778fffdb4e11593149eb7770071a0728653f19 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 18 Aug 2014 14:40:05 -0700 Subject: [PATCH 035/192] [mllib] DecisionTree: treeAggregate + Python example bug fix Small DecisionTree updates: * Changed main DecisionTree aggregate to treeAggregate. * Fixed bug in python example decision_tree_runner.py with missing argument (since categoricalFeaturesInfo is no longer an optional argument for trainClassifier). * Fixed same bug in python doc tests, and added tree.py to doc tests. CC: mengxr Author: Joseph K. Bradley Closes #2015 from jkbradley/dt-opt2 and squashes the following commits: b5114fa [Joseph K. Bradley] Fixed python tree.py doc test (extra newline) 8e4665d [Joseph K. Bradley] Added tree.py to python doc tests. Fixed bug from missing categoricalFeaturesInfo argument. b7b2922 [Joseph K. Bradley] Fixed bug in python example decision_tree_runner.py with missing argument. Changed main DecisionTree aggregate to treeAggregate. 85bbc1f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 66d076f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 a0ed0da [Joseph K. Bradley] Renamed DTMetadata to DecisionTreeMetadata. Small doc updates. 3726d20 [Joseph K. Bradley] Small code improvements based on code review. ac0b9f8 [Joseph K. Bradley] Small updates based on code review. Main change: Now using << instead of math.pow. db0d773 [Joseph K. Bradley] scala style fix 6a38f48 [Joseph K. Bradley] Added DTMetadata class for cleaner code 931a3a7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 797f68a [Joseph K. Bradley] Fixed DecisionTreeSuite bug for training second level. Needed to update treePointToNodeIndex with groupShift. f40381c [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 5f2dec2 [Joseph K. Bradley] Fixed scalastyle issue in TreePoint 6b5651e [Joseph K. Bradley] Updates based on code review. 1 major change: persisting to memory + disk, not just memory. 2d2aaaf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals (cherry picked from commit 115eeb30dd9c9dd10685a71f2c23ca23794d3142) Signed-off-by: Xiangrui Meng --- .../src/main/python/mllib/decision_tree_runner.py | 4 +++- .../org/apache/spark/mllib/tree/DecisionTree.scala | 3 ++- python/pyspark/mllib/tree.py | 14 ++++++++------ python/run-tests | 1 + 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 8efadb5223f56..db96a7cb3730f 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -124,7 +124,9 @@ def usage(): (reindexedData, origToNewLabels) = reindexClassLabels(points) # Train a classifier. - model = DecisionTree.trainClassifier(reindexedData, numClasses=2) + categoricalFeaturesInfo={} # no categorical features + model = DecisionTree.trainClassifier(reindexedData, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. print "Trained DecisionTree for classification:" print " Model numNodes: %d\n" % model.numNodes() 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 6b9a8f72c244e..5cdd258f6c20b 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 @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging +import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ @@ -826,7 +827,7 @@ object DecisionTree extends Serializable with Logging { // Calculate bin aggregates. timer.start("aggregation") val binAggregates = { - input.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) + input.treeAggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) } timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index e1a4671709b7d..e9d778df5a24b 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -88,7 +88,8 @@ class DecisionTree(object): It will probably be modified for Spark v1.2. Example usage: - >>> from numpy import array, ndarray + >>> from numpy import array + >>> import sys >>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.tree import DecisionTree >>> from pyspark.mllib.linalg import SparseVector @@ -99,15 +100,15 @@ class DecisionTree(object): ... LabeledPoint(1.0, [2.0]), ... LabeledPoint(1.0, [3.0]) ... ] - >>> - >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2) - >>> print(model) + >>> categoricalFeaturesInfo = {} # no categorical features + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2, + ... categoricalFeaturesInfo=categoricalFeaturesInfo) + >>> sys.stdout.write(model) DecisionTreeModel classifier If (feature 0 <= 0.5) Predict: 0.0 Else (feature 0 > 0.5) Predict: 1.0 - >>> model.predict(array([1.0])) > 0 True >>> model.predict(array([0.0])) == 0 @@ -119,7 +120,8 @@ class DecisionTree(object): ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> - >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data)) + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), + ... categoricalFeaturesInfo=categoricalFeaturesInfo) >>> model.predict(array([0.0, 1.0])) == 1 True >>> model.predict(array([0.0, 0.0])) == 0 diff --git a/python/run-tests b/python/run-tests index 1218edcbd7e08..a6271e0cf5fa9 100755 --- a/python/run-tests +++ b/python/run-tests @@ -79,6 +79,7 @@ run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" run_test "pyspark/mllib/tests.py" +run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" if [[ $FAILED == 0 ]]; then From e3f89e971b117e11d15e4b9b47e63da55f4e488b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 18 Aug 2014 18:01:39 -0700 Subject: [PATCH 036/192] [SPARK-2850] [SPARK-2626] [mllib] MLlib stats examples + small fixes Added examples for statistical summarization: * Scala: StatisticalSummary.scala ** Tests: correlation, MultivariateOnlineSummarizer * python: statistical_summary.py ** Tests: correlation (since MultivariateOnlineSummarizer has no Python API) Added examples for random and sampled RDDs: * Scala: RandomAndSampledRDDs.scala * python: random_and_sampled_rdds.py * Both test: ** RandomRDDGenerators.normalRDD, normalVectorRDD ** RDD.sample, takeSample, sampleByKey Added sc.stop() to all examples. CorrelationSuite.scala * Added 1 test for RDDs with only 1 value RowMatrix.scala * numCols(): Added check for numRows = 0, with error message. * computeCovariance(): Added check for numRows <= 1, with error message. Python SparseVector (pyspark/mllib/linalg.py) * Added toDense() function python/run-tests script * Added stat.py (doc test) CC: mengxr dorx Main changes were examples to show usage across APIs. Author: Joseph K. Bradley Closes #1878 from jkbradley/mllib-stats-api-check and squashes the following commits: ea5c047 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check dafebe2 [Joseph K. Bradley] Bug fixes for examples SampledRDDs.scala and sampled_rdds.py: Check for division by 0 and for missing key in maps. 8d1e555 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 60c72d9 [Joseph K. Bradley] Fixed stat.py doc test to work for Python versions printing nan or NaN. b20d90a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 4e5d15e [Joseph K. Bradley] Changed pyspark/mllib/stat.py doc tests to use NaN instead of nan. 32173b7 [Joseph K. Bradley] Stats examples update. c8c20dc [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check cf70b07 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 0b7cec3 [Joseph K. Bradley] Small updates based on code review. Renamed statistical_summary.py to correlations.py ab48f6e [Joseph K. Bradley] RowMatrix.scala * numCols(): Added check for numRows = 0, with error message. * computeCovariance(): Added check for numRows <= 1, with error message. 65e4ebc [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 8195c78 [Joseph K. Bradley] Added examples for random and sampled RDDs: * Scala: RandomAndSampledRDDs.scala * python: random_and_sampled_rdds.py * Both test: ** RandomRDDGenerators.normalRDD, normalVectorRDD ** RDD.sample, takeSample, sampleByKey 064985b [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check ee918e9 [Joseph K. Bradley] Added examples for statistical summarization: * Scala: StatisticalSummary.scala ** Tests: correlation, MultivariateOnlineSummarizer * python: statistical_summary.py ** Tests: correlation (since MultivariateOnlineSummarizer has no Python API) (cherry picked from commit c8b16ca0d86cc60fb960eebf0cb383f159a88b03) Signed-off-by: Xiangrui Meng --- examples/src/main/python/als.py | 2 + .../src/main/python/cassandra_inputformat.py | 2 + .../src/main/python/cassandra_outputformat.py | 2 + examples/src/main/python/hbase_inputformat.py | 2 + .../src/main/python/hbase_outputformat.py | 2 + examples/src/main/python/kmeans.py | 2 + .../src/main/python/logistic_regression.py | 2 + .../src/main/python/mllib/correlations.py | 60 +++++++++ .../main/python/mllib/decision_tree_runner.py | 5 + examples/src/main/python/mllib/kmeans.py | 1 + .../main/python/mllib/logistic_regression.py | 1 + .../python/mllib/random_rdd_generation.py | 55 ++++++++ .../src/main/python/mllib/sampled_rdds.py | 86 ++++++++++++ examples/src/main/python/pagerank.py | 2 + examples/src/main/python/pi.py | 2 + examples/src/main/python/sort.py | 2 + .../src/main/python/transitive_closure.py | 2 + examples/src/main/python/wordcount.py | 2 + .../spark/examples/mllib/Correlations.scala | 92 +++++++++++++ .../mllib/MultivariateSummarizer.scala | 98 ++++++++++++++ .../examples/mllib/RandomRDDGeneration.scala | 60 +++++++++ .../spark/examples/mllib/SampledRDDs.scala | 126 ++++++++++++++++++ .../mllib/linalg/distributed/RowMatrix.scala | 14 +- .../stat/MultivariateOnlineSummarizer.scala | 8 +- .../spark/mllib/stat/CorrelationSuite.scala | 15 ++- .../MultivariateOnlineSummarizerSuite.scala | 6 +- python/pyspark/mllib/linalg.py | 10 ++ python/pyspark/mllib/stat.py | 22 +-- python/run-tests | 1 + 29 files changed, 664 insertions(+), 20 deletions(-) create mode 100755 examples/src/main/python/mllib/correlations.py create mode 100755 examples/src/main/python/mllib/random_rdd_generation.py create mode 100755 examples/src/main/python/mllib/sampled_rdds.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index c862650b0aa1d..5b1fa4d997eeb 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -97,3 +97,5 @@ def update(i, vec, mat, ratings): error = rmse(R, ms, us) print "Iteration %d:" % i print "\nRMSE: %5.4f\n" % error + + sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 39fa6b0d22ef5..e4a897f61e39d 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -77,3 +77,5 @@ output = cass_rdd.collect() for (k, v) in output: print (k, v) + + sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index 1dfbf98604425..836c35b5c6794 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -81,3 +81,5 @@ conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", valueConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLValueConverter") + + sc.stop() diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index c9fa8e171c2a1..befacee0dea56 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -71,3 +71,5 @@ output = hbase_rdd.collect() for (k, v) in output: print (k, v) + + sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index 5e11548fd13f7..49bbc5aebdb0b 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -63,3 +63,5 @@ conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") + + sc.stop() diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 036bdf4c4f999..86ef6f32c84e8 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -77,3 +77,5 @@ def closestPoint(p, centers): kPoints[x] = y print "Final centers: " + str(kPoints) + + sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 8456b272f9c05..3aa56b0528168 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -80,3 +80,5 @@ def add(x, y): w -= points.map(lambda m: gradient(m, w)).reduce(add) print "Final w: " + str(w) + + sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py new file mode 100755 index 0000000000000..6b16a56e44af7 --- /dev/null +++ b/examples/src/main/python/mllib/correlations.py @@ -0,0 +1,60 @@ +# +# 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. +# + +""" +Correlations using MLlib. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.stat import Statistics +from pyspark.mllib.util import MLUtils + + +if __name__ == "__main__": + if len(sys.argv) not in [1,2]: + print >> sys.stderr, "Usage: correlations ()" + exit(-1) + sc = SparkContext(appName="PythonCorrelations") + if len(sys.argv) == 2: + filepath = sys.argv[1] + else: + filepath = 'data/mllib/sample_linear_regression_data.txt' + corrType = 'pearson' + + points = MLUtils.loadLibSVMFile(sc, filepath)\ + .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) + + print + print 'Summary of data file: ' + filepath + print '%d data points' % points.count() + + # Statistics (correlations) + print + print 'Correlation (%s) between label and each feature' % corrType + print 'Feature\tCorrelation' + numFeatures = points.take(1)[0].features.size + labelRDD = points.map(lambda lp: lp.label) + for i in range(numFeatures): + featureRDD = points.map(lambda lp: lp.features[i]) + corr = Statistics.corr(labelRDD, featureRDD, corrType) + print '%d\t%g' % (i, corr) + print + + sc.stop() diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index db96a7cb3730f..6e4a4a0cb6be0 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -17,6 +17,8 @@ """ Decision tree classification and regression using MLlib. + +This example requires NumPy (http://www.numpy.org/). """ import numpy, os, sys @@ -117,6 +119,7 @@ def usage(): if len(sys.argv) == 2: dataPath = sys.argv[1] if not os.path.isfile(dataPath): + sc.stop() usage() points = MLUtils.loadLibSVMFile(sc, dataPath) @@ -133,3 +136,5 @@ def usage(): print " Model depth: %d\n" % model.depth() print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) print model + + sc.stop() diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index b308132c9aeeb..2eeb1abeeb12b 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -42,3 +42,4 @@ def parseVector(line): k = int(sys.argv[2]) model = KMeans.train(data, k) print "Final centers: " + str(model.clusterCenters) + sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 9d547ff77c984..8cae27fc4a52d 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -50,3 +50,4 @@ def parsePoint(line): model = LogisticRegressionWithSGD.train(points, iterations) print "Final weights: " + str(model.weights) print "Final intercept: " + str(model.intercept) + sc.stop() diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py new file mode 100755 index 0000000000000..b388d8d83fb86 --- /dev/null +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -0,0 +1,55 @@ +# +# 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. +# + +""" +Randomly generated RDDs. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.random import RandomRDDs + + +if __name__ == "__main__": + if len(sys.argv) not in [1, 2]: + print >> sys.stderr, "Usage: random_rdd_generation" + exit(-1) + + sc = SparkContext(appName="PythonRandomRDDGeneration") + + numExamples = 10000 # number of examples to generate + fraction = 0.1 # fraction of data to sample + + # Example: RandomRDDs.normalRDD + normalRDD = RandomRDDs.normalRDD(sc, numExamples) + print 'Generated RDD of %d examples sampled from the standard normal distribution'\ + % normalRDD.count() + print ' First 5 samples:' + for sample in normalRDD.take(5): + print ' ' + str(sample) + print + + # Example: RandomRDDs.normalVectorRDD + normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() + print ' First 5 samples:' + for sample in normalVectorRDD.take(5): + print ' ' + str(sample) + print + + sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py new file mode 100755 index 0000000000000..ec64a5978c672 --- /dev/null +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -0,0 +1,86 @@ +# +# 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. +# + +""" +Randomly sampled RDDs. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.util import MLUtils + + +if __name__ == "__main__": + if len(sys.argv) not in [1, 2]: + print >> sys.stderr, "Usage: sampled_rdds " + exit(-1) + if len(sys.argv) == 2: + datapath = sys.argv[1] + else: + datapath = 'data/mllib/sample_binary_classification_data.txt' + + sc = SparkContext(appName="PythonSampledRDDs") + + fraction = 0.1 # fraction of data to sample + + examples = MLUtils.loadLibSVMFile(sc, datapath) + numExamples = examples.count() + if numExamples == 0: + print >> sys.stderr, "Error: Data file had no samples to load." + exit(1) + print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + + # Example: RDD.sample() and RDD.takeSample() + expectedSampleSize = int(numExamples * fraction) + print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ + % (fraction, expectedSampleSize) + sampledRDD = examples.sample(withReplacement = True, fraction = fraction) + print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + sampledArray = examples.takeSample(withReplacement = True, num = expectedSampleSize) + print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + + print + + # Example: RDD.sampleByKey() + keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) + print ' Keyed data using label (Int) as key ==> Orig' + # Count examples per label in original data. + keyCountsA = keyedRDD.countByKey() + + # Subsample, and count examples per label in sampled data. + fractions = {} + for k in keyCountsA.keys(): + fractions[k] = fraction + sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = True, fractions = fractions) + keyCountsB = sampledByKeyRDD.countByKey() + sizeB = sum(keyCountsB.values()) + print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ + % sizeB + + # Compare samples + print ' \tFractions of examples with key' + print 'Key\tOrig\tSample' + for k in sorted(keyCountsA.keys()): + fracA = keyCountsA[k] / float(numExamples) + if sizeB != 0: + fracB = keyCountsB.get(k, 0) / float(sizeB) + else: + fracB = 0 + print '%d\t%g\t%g' % (k, fracA, fracB) + + sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 0b96343158d44..b539c4128cdcc 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -68,3 +68,5 @@ def parseNeighbors(urls): # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): print "%s has rank: %s." % (link, rank) + + sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 21d94a2cd4b64..fc37459dc74aa 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -37,3 +37,5 @@ def f(_): count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) + + sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index 41d00c1b79133..bb686f17518a0 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -34,3 +34,5 @@ output = sortedCount.collect() for (num, unitcount) in output: print num + + sc.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 8698369b13d84..bf331b542c438 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -64,3 +64,5 @@ def generateGraph(): break print "TC has %i edges" % tc.count() + + sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index dcc095fdd0ed9..ae6cd13b83d92 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -33,3 +33,5 @@ output = counts.collect() for (word, count) in output: print "%s: %i" % (word, count) + + sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala new file mode 100644 index 0000000000000..d6b2fe430e5a4 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -0,0 +1,92 @@ +/* + * 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.mllib.stat.Statistics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.{SparkConf, SparkContext} + + +/** + * An example app for summarizing multivariate data from a file. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.Correlations + * }}} + * By default, this loads a synthetic dataset from `data/mllib/sample_linear_regression_data.txt`. + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object Correlations { + + case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + + def main(args: Array[String]) { + + val defaultParams = Params() + + val parser = new OptionParser[Params]("Correlations") { + head("Correlations: an example app for computing correlations") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.Correlations \ + | examples/target/scala-*/spark-examples-*.jar \ + | --input data/mllib/sample_linear_regression_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"Correlations with $params") + val sc = new SparkContext(conf) + + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() + + println(s"Summary of data file: ${params.input}") + println(s"${examples.count()} data points") + + // Calculate label -- feature correlations + val labelRDD = examples.map(_.label) + val numFeatures = examples.take(1)(0).features.size + val corrType = "pearson" + println() + println(s"Correlation ($corrType) between label and each feature") + println(s"Feature\tCorrelation") + var feature = 0 + while (feature < numFeatures) { + val featureRDD = examples.map(_.features(feature)) + val corr = Statistics.corr(labelRDD, featureRDD) + println(s"$feature\t$corr") + feature += 1 + } + println() + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala new file mode 100644 index 0000000000000..4532512c01f84 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -0,0 +1,98 @@ +/* + * 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.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.{SparkConf, SparkContext} + + +/** + * An example app for summarizing multivariate data from a file. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.MultivariateSummarizer + * }}} + * By default, this loads a synthetic dataset from `data/mllib/sample_linear_regression_data.txt`. + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object MultivariateSummarizer { + + case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + + def main(args: Array[String]) { + + val defaultParams = Params() + + val parser = new OptionParser[Params]("MultivariateSummarizer") { + head("MultivariateSummarizer: an example app for MultivariateOnlineSummarizer") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.MultivariateSummarizer \ + | examples/target/scala-*/spark-examples-*.jar \ + | --input data/mllib/sample_linear_regression_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"MultivariateSummarizer with $params") + val sc = new SparkContext(conf) + + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() + + println(s"Summary of data file: ${params.input}") + println(s"${examples.count()} data points") + + // Summarize labels + val labelSummary = examples.aggregate(new MultivariateOnlineSummarizer())( + (summary, lp) => summary.add(Vectors.dense(lp.label)), + (sum1, sum2) => sum1.merge(sum2)) + + // Summarize features + val featureSummary = examples.aggregate(new MultivariateOnlineSummarizer())( + (summary, lp) => summary.add(lp.features), + (sum1, sum2) => sum1.merge(sum2)) + + println() + println(s"Summary statistics") + println(s"\tLabel\tFeatures") + println(s"mean\t${labelSummary.mean(0)}\t${featureSummary.mean.toArray.mkString("\t")}") + println(s"var\t${labelSummary.variance(0)}\t${featureSummary.variance.toArray.mkString("\t")}") + println( + s"nnz\t${labelSummary.numNonzeros(0)}\t${featureSummary.numNonzeros.toArray.mkString("\t")}") + println(s"max\t${labelSummary.max(0)}\t${featureSummary.max.toArray.mkString("\t")}") + println(s"min\t${labelSummary.min(0)}\t${featureSummary.min.toArray.mkString("\t")}") + println() + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala new file mode 100644 index 0000000000000..924b586e3af99 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala @@ -0,0 +1,60 @@ +/* + * 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.spark.mllib.random.RandomRDDs +import org.apache.spark.rdd.RDD + +import org.apache.spark.{SparkConf, SparkContext} + +/** + * An example app for randomly generated RDDs. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.RandomRDDGeneration + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object RandomRDDGeneration { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName(s"RandomRDDGeneration") + val sc = new SparkContext(conf) + + val numExamples = 10000 // number of examples to generate + val fraction = 0.1 // fraction of data to sample + + // Example: RandomRDDs.normalRDD + val normalRDD: RDD[Double] = RandomRDDs.normalRDD(sc, numExamples) + println(s"Generated RDD of ${normalRDD.count()}" + + " examples sampled from the standard normal distribution") + println(" First 5 samples:") + normalRDD.take(5).foreach( x => println(s" $x") ) + + // Example: RandomRDDs.normalVectorRDD + val normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + println(s"Generated RDD of ${normalVectorRDD.count()} examples of length-2 vectors.") + println(" First 5 samples:") + normalVectorRDD.take(5).foreach( x => println(s" $x") ) + + println() + + sc.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala new file mode 100644 index 0000000000000..f01b8266e3fe3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -0,0 +1,126 @@ +/* + * 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.spark.mllib.util.MLUtils +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ + +/** + * An example app for randomly generated and sampled RDDs. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.SampledRDDs + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object SampledRDDs { + + case class Params(input: String = "data/mllib/sample_binary_classification_data.txt") + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("SampledRDDs") { + head("SampledRDDs: an example app for randomly generated and sampled RDDs.") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.SampledRDDs \ + | examples/target/scala-*/spark-examples-*.jar + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"SampledRDDs with $params") + val sc = new SparkContext(conf) + + val fraction = 0.1 // fraction of data to sample + + val examples = MLUtils.loadLibSVMFile(sc, params.input) + val numExamples = examples.count() + if (numExamples == 0) { + throw new RuntimeException("Error: Data file had no samples to load.") + } + println(s"Loaded data with $numExamples examples from file: ${params.input}") + + // Example: RDD.sample() and RDD.takeSample() + val expectedSampleSize = (numExamples * fraction).toInt + println(s"Sampling RDD using fraction $fraction. Expected sample size = $expectedSampleSize.") + val sampledRDD = examples.sample(withReplacement = true, fraction = fraction) + println(s" RDD.sample(): sample has ${sampledRDD.count()} examples") + val sampledArray = examples.takeSample(withReplacement = true, num = expectedSampleSize) + println(s" RDD.takeSample(): sample has ${sampledArray.size} examples") + + println() + + // Example: RDD.sampleByKey() and RDD.sampleByKeyExact() + val keyedRDD = examples.map { lp => (lp.label.toInt, lp.features) } + println(s" Keyed data using label (Int) as key ==> Orig") + // Count examples per label in original data. + val keyCounts = keyedRDD.countByKey() + + // Subsample, and count examples per label in sampled data. (approximate) + val fractions = keyCounts.keys.map((_, fraction)).toMap + val sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = true, fractions = fractions) + val keyCountsB = sampledByKeyRDD.countByKey() + val sizeB = keyCountsB.values.sum + println(s" Sampled $sizeB examples using approximate stratified sampling (by label)." + + " ==> Approx Sample") + + // Subsample, and count examples per label in sampled data. (approximate) + val sampledByKeyRDDExact = + keyedRDD.sampleByKeyExact(withReplacement = true, fractions = fractions) + val keyCountsBExact = sampledByKeyRDDExact.countByKey() + val sizeBExact = keyCountsBExact.values.sum + println(s" Sampled $sizeBExact examples using exact stratified sampling (by label)." + + " ==> Exact Sample") + + // Compare samples + println(s" \tFractions of examples with key") + println(s"Key\tOrig\tApprox Sample\tExact Sample") + keyCounts.keys.toSeq.sorted.foreach { key => + val origFrac = keyCounts(key) / numExamples.toDouble + val approxFrac = if (sizeB != 0) { + keyCountsB.getOrElse(key, 0L) / sizeB.toDouble + } else { + 0 + } + val exactFrac = if (sizeBExact != 0) { + keyCountsBExact.getOrElse(key, 0L) / sizeBExact.toDouble + } else { + 0 + } + println(s"$key\t$origFrac\t$approxFrac\t$exactFrac") + } + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index e76bc9fefff01..2e414a73be8e0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -53,8 +53,14 @@ class RowMatrix( /** Gets or computes the number of columns. */ override def numCols(): Long = { if (nCols <= 0) { - // Calling `first` will throw an exception if `rows` is empty. - nCols = rows.first().size + try { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size + } catch { + case err: UnsupportedOperationException => + sys.error("Cannot determine the number of cols because it is not specified in the " + + "constructor and the rows RDD is empty.") + } } nCols } @@ -293,6 +299,10 @@ class RowMatrix( (s1._1 + s2._1, s1._2 += s2._2) ) + if (m <= 1) { + sys.error(s"RowMatrix.computeCovariance called on matrix with only $m rows." + + " Cannot compute the covariance of a RowMatrix with <= 1 row.") + } updateNumRows(m) mean :/= m.toDouble diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 5105b5c37aaaa..7d845c44365dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -55,8 +55,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ def add(sample: Vector): this.type = { if (n == 0) { - require(sample.toBreeze.length > 0, s"Vector should have dimension larger than zero.") - n = sample.toBreeze.length + require(sample.size > 0, s"Vector should have dimension larger than zero.") + n = sample.size currMean = BDV.zeros[Double](n) currM2n = BDV.zeros[Double](n) @@ -65,8 +65,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMin = BDV.fill(n)(Double.MaxValue) } - require(n == sample.toBreeze.length, s"Dimensions mismatch when adding new sample." + - s" Expecting $n but got ${sample.toBreeze.length}.") + require(n == sample.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $n but got ${sample.size}.") sample.toBreeze.activeIterator.foreach { case (_, 0.0) => // Skip explicit zero elements. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index a3f76f77a5dcc..34548c86ebc14 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -39,6 +39,17 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { Vectors.dense(9.0, 0.0, 0.0, 1.0) ) + test("corr(x, y) pearson, 1 value in data") { + val x = sc.parallelize(Array(1.0)) + val y = sc.parallelize(Array(4.0)) + intercept[RuntimeException] { + Statistics.corr(x, y, "pearson") + } + intercept[RuntimeException] { + Statistics.corr(x, y, "spearman") + } + } + test("corr(x, y) default, pearson") { val x = sc.parallelize(xData) val y = sc.parallelize(yData) @@ -58,7 +69,7 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { // RDD of zero variance val z = sc.parallelize(zeros) - assert(Statistics.corr(x, z).isNaN()) + assert(Statistics.corr(x, z).isNaN) } test("corr(x, y) spearman") { @@ -78,7 +89,7 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { // RDD of zero variance => zero variance in ranks val z = sc.parallelize(zeros) - assert(Statistics.corr(x, z, "spearman").isNaN()) + assert(Statistics.corr(x, z, "spearman").isNaN) } test("corr(X) default, pearson") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala index db13f142df517..1e9415249104b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala @@ -139,7 +139,8 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") assert(summarizer.variance ~== - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") assert(summarizer.count === 6) } @@ -167,7 +168,8 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") assert(summarizer.variance ~== - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") assert(summarizer.count === 6) } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 9a239abfbbeb1..f485a69db1fa2 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -23,6 +23,7 @@ SciPy is available in their environment. """ +import numpy from numpy import array, array_equal, ndarray, float64, int32 @@ -160,6 +161,15 @@ def squared_distance(self, other): j += 1 return result + def toArray(self): + """ + Returns a copy of this SparseVector as a 1-dimensional NumPy array. + """ + arr = numpy.zeros(self.size) + for i in xrange(self.indices.size): + arr[self.indices[i]] = self.values[i] + return arr + def __str__(self): inds = "[" + ",".join([str(i) for i in self.indices]) + "]" vals = "[" + ",".join([str(v) for v in self.values]) + "]" diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index a73abc5ff90df..feef0d16cd644 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -118,16 +118,18 @@ def corr(x, y=None, method=None): >>> from linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) - >>> Statistics.corr(rdd) - array([[ 1. , 0.05564149, nan, 0.40047142], - [ 0.05564149, 1. , nan, 0.91359586], - [ nan, nan, 1. , nan], - [ 0.40047142, 0.91359586, nan, 1. ]]) - >>> Statistics.corr(rdd, method="spearman") - array([[ 1. , 0.10540926, nan, 0.4 ], - [ 0.10540926, 1. , nan, 0.9486833 ], - [ nan, nan, 1. , nan], - [ 0.4 , 0.9486833 , nan, 1. ]]) + >>> pearsonCorr = Statistics.corr(rdd) + >>> print str(pearsonCorr).replace('nan', 'NaN') + [[ 1. 0.05564149 NaN 0.40047142] + [ 0.05564149 1. NaN 0.91359586] + [ NaN NaN 1. NaN] + [ 0.40047142 0.91359586 NaN 1. ]] + >>> spearmanCorr = Statistics.corr(rdd, method="spearman") + >>> print str(spearmanCorr).replace('nan', 'NaN') + [[ 1. 0.10540926 NaN 0.4 ] + [ 0.10540926 1. NaN 0.9486833 ] + [ NaN NaN 1. NaN] + [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") ... print "Method name as second argument without 'method=' shouldn't be allowed." diff --git a/python/run-tests b/python/run-tests index a6271e0cf5fa9..b506559a5e810 100755 --- a/python/run-tests +++ b/python/run-tests @@ -78,6 +78,7 @@ run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" +run_test "pyspark/mllib/stat.py" run_test "pyspark/mllib/tests.py" run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" From 7d069bf0c57b75b53b449fcc51cf7fd616f8686d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 18 Aug 2014 18:20:54 -0700 Subject: [PATCH 037/192] [SPARK-3108][MLLIB] add predictOnValues to StreamingLR and fix predictOn It is useful in streaming to allow users to carry extra data with the prediction, for monitoring the prediction error for example. freeman-lab Author: Xiangrui Meng Closes #2023 from mengxr/predict-on-values and squashes the following commits: cac47b8 [Xiangrui Meng] add classtag 2821b3b [Xiangrui Meng] use mapValues 0925efa [Xiangrui Meng] add predictOnValues to StreamingLR and fix predictOn (cherry picked from commit 217b5e915e2f21f047dfc4be680cd20d58baf9f8) Signed-off-by: Xiangrui Meng --- .../mllib/StreamingLinearRegression.scala | 4 +-- .../regression/StreamingLinearAlgorithm.scala | 31 +++++++++++++++---- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index 0e992fa9967bb..c5bd5b0b178d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -59,10 +59,10 @@ object StreamingLinearRegression { val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() - .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) + .setInitialWeights(Vectors.zeros(args(3).toInt)) model.trainOn(trainingData) - model.predictOn(testData).print() + model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() ssc.start() ssc.awaitTermination() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index b8b0b42611775..8db0442a7a569 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -17,8 +17,12 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.DeveloperApi +import scala.reflect.ClassTag + import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** @@ -92,15 +96,30 @@ abstract class StreamingLinearAlgorithm[ /** * Use the model to make predictions on batches of data from a DStream * - * @param data DStream containing labeled data + * @param data DStream containing feature vectors * @return DStream containing predictions */ - def predictOn(data: DStream[LabeledPoint]): DStream[Double] = { + def predictOn(data: DStream[Vector]): DStream[Double] = { if (Option(model.weights) == None) { - logError("Initial weights must be set before starting prediction") - throw new IllegalArgumentException + val msg = "Initial weights must be set before starting prediction" + logError(msg) + throw new IllegalArgumentException(msg) } - data.map(x => model.predict(x.features)) + data.map(model.predict) } + /** + * Use the model to make predictions on the values of a DStream and carry over its keys. + * @param data DStream containing feature vectors + * @tparam K key type + * @return DStream containing the input keys and the predictions as values + */ + def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Double)] = { + if (Option(model.weights) == None) { + val msg = "Initial weights must be set before starting prediction" + logError(msg) + throw new IllegalArgumentException(msg) + } + data.mapValues(model.predict) + } } From 3a03259a0421b08269a3b23cab2bdb4f9108f5c5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 18 Aug 2014 20:42:19 -0700 Subject: [PATCH 038/192] [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL. This fixes SPARK-3114, an issue where we inadvertently broke Python UDFs in Spark SQL. This PR modifiers the test runner script to always run the PySpark SQL tests, irrespective of whether SparkSQL itself has been modified. It also includes Davies' fix for the bug. Closes #2026. Author: Josh Rosen Author: Davies Liu Closes #2027 from JoshRosen/pyspark-sql-fix and squashes the following commits: 9af2708 [Davies Liu] bugfix: disable compression of command 0d8d3a4 [Josh Rosen] Always run Python Spark SQL tests. (cherry picked from commit 1f1819b20f887b487557c31e54b8bcd95b582dc6) Signed-off-by: Josh Rosen --- dev/run-tests | 17 +++++++++++++---- python/pyspark/rdd.py | 2 +- python/pyspark/worker.py | 2 +- python/run-tests | 4 +--- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 0e24515d1376c..132f696d6447a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -58,7 +58,7 @@ if [ -n "$AMPLAB_JENKINS" ]; then diffs=`git diff --name-only master | grep "^sql/"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." - export _RUN_SQL_TESTS=true # exported for PySpark tests + _RUN_SQL_TESTS=true fi fi @@ -89,13 +89,22 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" +# Build Spark; we always build with Hive because the PySpark SparkSQL tests need it. +# echo "q" is needed because sbt on encountering a build file with failure +# (either resolution or compilation) prompts the user for input either q, r, +# etc to quit or retry. This echo is there to make it not block. +BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver " +echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + +# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, +# echo "q" is needed because sbt on encountering a build file with failure +# (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS clean package assembly/assembly test | \ +echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS test | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "" diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c708b69cc1e31..86cd89b245aea 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1812,7 +1812,7 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) - ser = CompressedSerializer(CloudPickleSerializer()) + ser = CloudPickleSerializer() pickled_command = ser.dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 77a9c4a0e0677..6805063e06798 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -72,7 +72,7 @@ def main(infile, outfile): value = ser._read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, value) - command = ser._read_with_length(infile) + command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) diff --git a/python/run-tests b/python/run-tests index b506559a5e810..7b1ee3e1cddba 100755 --- a/python/run-tests +++ b/python/run-tests @@ -59,9 +59,7 @@ $PYSPARK_PYTHON --version run_test "pyspark/rdd.py" run_test "pyspark/context.py" run_test "pyspark/conf.py" -if [ -n "$_RUN_SQL_TESTS" ]; then - run_test "pyspark/sql.py" -fi +run_test "pyspark/sql.py" # These tests are included in the module-level docs, and so must # be handled on a higher level rather than within the python file. export PYSPARK_DOC_TEST=1 From b6d8e66517f264e8576c785624fee9d1ff26900b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Aug 2014 20:51:41 -0700 Subject: [PATCH 039/192] [SPARK-3116] Remove the excessive lockings in TorrentBroadcast Author: Reynold Xin Closes #2028 from rxin/torrentBroadcast and squashes the following commits: 92c62a5 [Reynold Xin] Revert the MEMORY_AND_DISK_SER changes. 03a5221 [Reynold Xin] [SPARK-3116] Remove the excessive lockings in TorrentBroadcast (cherry picked from commit 82577339dd58b5811eab5d10667775e61e37ff51) Signed-off-by: Reynold Xin --- .../spark/broadcast/TorrentBroadcast.scala | 66 ++++++++----------- 1 file changed, 27 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index fe73456ef8fad..d8be649f96e5f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,8 +17,7 @@ package org.apache.spark.broadcast -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, InputStream, - ObjectInputStream, ObjectOutputStream, OutputStream} +import java.io._ import scala.reflect.ClassTag import scala.util.Random @@ -53,10 +52,8 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private val broadcastId = BroadcastBlockId(id) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - } + SparkEnv.get.blockManager.putSingle( + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) @transient private var arrayOfBlocks: Array[TorrentBlock] = null @transient private var totalBlocks = -1 @@ -91,18 +88,14 @@ private[spark] class TorrentBroadcast[T: ClassTag]( // Store meta-info val metaId = BroadcastBlockId(id, "meta") val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } + SparkEnv.get.blockManager.putSingle( + metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // Store individual pieces for (i <- 0 until totalBlocks) { val pieceId = BroadcastBlockId(id, "piece" + i) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } + SparkEnv.get.blockManager.putSingle( + pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) } } @@ -165,21 +158,20 @@ private[spark] class TorrentBroadcast[T: ClassTag]( val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 while (attemptId > 0 && totalBlocks == -1) { - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => - val tInfo = x.asInstanceOf[TorrentInfo] - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - arrayOfBlocks = new Array[TorrentBlock](totalBlocks) - hasBlocks = 0 - - case None => - Thread.sleep(500) - } + SparkEnv.get.blockManager.getSingle(metaId) match { + case Some(x) => + val tInfo = x.asInstanceOf[TorrentInfo] + totalBlocks = tInfo.totalBlocks + totalBytes = tInfo.totalBytes + arrayOfBlocks = new Array[TorrentBlock](totalBlocks) + hasBlocks = 0 + + case None => + Thread.sleep(500) } attemptId -= 1 } + if (totalBlocks == -1) { return false } @@ -192,17 +184,15 @@ private[spark] class TorrentBroadcast[T: ClassTag]( val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) for (pid <- recvOrder) { val pieceId = BroadcastBlockId(id, "piece" + pid) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => - arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] - hasBlocks += 1 - SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) + SparkEnv.get.blockManager.getSingle(pieceId) match { + case Some(x) => + arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] + hasBlocks += 1 + SparkEnv.get.blockManager.putSingle( + pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - case None => - throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) - } + case None => + throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) } } @@ -291,9 +281,7 @@ private[broadcast] object TorrentBroadcast extends Logging { * If removeFromDriver is true, also remove these persisted blocks on the driver. */ def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { - synchronized { - SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) - } + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } } From f3b0f34b4403bceb9b98740084b9ddba4314d71a Mon Sep 17 00:00:00 2001 From: Matt Forbes Date: Mon, 18 Aug 2014 21:43:32 -0700 Subject: [PATCH 040/192] Fix typo in decision tree docs Candidate splits were inconsistent with the example. Author: Matt Forbes Closes #1837 from emef/tree-doc and squashes the following commits: 3be14a1 [Matt Forbes] Fix typo in decision tree docs (cherry picked from commit cd0720ca77894d481fb73a8b5bb517013843cb1e) Signed-off-by: Xiangrui Meng --- docs/mllib-decision-tree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 9cbd880897578..c01a92a9a1b26 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -84,8 +84,8 @@ Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B -and A , B \| C where \| denotes the split. A similar heuristic is used for multiclass classification +features are ordered as A followed by C followed B or A, C, B. The two split candidates are A \| C, B +and A , C \| B where \| denotes the split. A similar heuristic is used for multiclass classification when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value is used for ordering. From 1418893da557892b86fc47f1e41e91880d4f8eda Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 19 Aug 2014 09:40:31 -0500 Subject: [PATCH 041/192] [SPARK-3072] YARN - Exit when reach max number failed executors In some cases on hadoop 2.x the spark application master doesn't properly exit and hangs around for 10 minutes after its really done. We should make sure it exits properly and stops the driver. Author: Thomas Graves Closes #2022 from tgravescs/SPARK-3072 and squashes the following commits: 665701d [Thomas Graves] Exit when reach max number failed executors (cherry picked from commit 7eb9cbc273d758522e787fcb2ef68ef65911475f) Signed-off-by: Thomas Graves --- .../spark/deploy/yarn/ApplicationMaster.scala | 33 ++++++++++++------- .../spark/deploy/yarn/ExecutorLauncher.scala | 5 +-- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++--- .../spark/deploy/yarn/ExecutorLauncher.scala | 5 +-- 4 files changed, 40 insertions(+), 19 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 62b5c3bc5f0f3..46a01f5a9a2cc 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -267,12 +267,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - // Exists the loop if the user thread exits. - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } + // Exits the loop if the user thread exits. + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive + && !isFinished) { + checkNumExecutorsFailed() yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) @@ -303,11 +301,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { - while (userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } + while (userThread.isAlive && !isFinished) { + checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". @@ -327,6 +322,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + logInfo("max number of executor failures reached") + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + // make sure to stop the user thread + val sparkContext = ApplicationMaster.sparkContextRef.get() + if (sparkContext != null) { + logInfo("Invoking sc stop from checkNumExecutorsFailed") + sparkContext.stop() + } else { + logError("sparkContext is null when should shutdown") + } + } + } + private def sendProgress() { logDebug("Sending progress") // Simulated with an allocate request with no nodes requested ... diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 184e2ad6c82cd..72c7143edcd71 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -249,7 +249,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed) && + !isFinished) { yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) checkNumExecutorsFailed() @@ -271,7 +272,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { - while (!driverClosed) { + while (!driverClosed && !isFinished) { checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 035356d390c80..9c2bcf17a8508 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -247,13 +247,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, yarnAllocator.allocateResources() // Exits the loop if the user thread exits. - var iters = 0 - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive + && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) - iters += 1 } } logInfo("All executors have launched.") @@ -271,8 +270,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def checkNumExecutorsFailed() { if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + logInfo("max number of executor failures reached") finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of executor failures reached") + // make sure to stop the user thread + val sparkContext = ApplicationMaster.sparkContextRef.get() + if (sparkContext != null) { + logInfo("Invoking sc stop from checkNumExecutorsFailed") + sparkContext.stop() + } else { + logError("sparkContext is null when should shutdown") + } } } @@ -289,7 +297,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { - while (userThread.isAlive) { + while (userThread.isAlive && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index fc7b8320d734d..a7585748b7f88 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -217,7 +217,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp // Wait until all containers have launched yarnAllocator.addResourceRequests(args.numExecutors) yarnAllocator.allocateResources() - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed) && + !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() @@ -249,7 +250,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { - while (!driverClosed) { + while (!driverClosed && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") From 5d895ad5668823a52b143ac39d9ffa264fc2a7b2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 19 Aug 2014 10:15:11 -0700 Subject: [PATCH 042/192] [SPARK-3089] Fix meaningless error message in ConnectionManager Author: Kousuke Saruta Closes #2000 from sarutak/SPARK-3089 and squashes the following commits: 02dfdea [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3089 e759ce7 [Kousuke Saruta] Improved error message when closing SendingConnection (cherry picked from commit cbfc26ba45f49559e64276c72e3054c6fe30ddd5) Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index e77d762bdf221..b3e951ded6e77 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -467,7 +467,7 @@ private[spark] class ConnectionManager( val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId) if (!sendingConnectionOpt.isDefined) { - logError("Corresponding SendingConnectionManagerId not found") + logError(s"Corresponding SendingConnection to ${remoteConnectionManagerId} not found") return } From 04a32086212452d3488e12dd64ffa18af0243345 Mon Sep 17 00:00:00 2001 From: freeman Date: Tue, 19 Aug 2014 13:28:57 -0700 Subject: [PATCH 043/192] [SPARK-3128][MLLIB] Use streaming test suite for StreamingLR Refactored tests for streaming linear regression to use existing streaming test utilities. Summary of changes: - Made ``mllib`` depend on tests from ``streaming`` - Rewrote accuracy and convergence tests to use ``setupStreams`` and ``runStreams`` - Added new test for the accuracy of predictions generated by ``predictOnValue`` These tests should run faster, be easier to extend/maintain, and provide a reference for new tests. mengxr tdas Author: freeman Closes #2037 from freeman-lab/streamingLR-predict-tests and squashes the following commits: e851ca7 [freeman] Fixed long lines 50eb0bf [freeman] Refactored tests to use streaming test tools 32c43c2 [freeman] Added test for prediction (cherry picked from commit 31f0b071efd0b63eb9d6a6a131e5c4fa28237583) Signed-off-by: Tathagata Das --- mllib/pom.xml | 7 + .../StreamingLinearRegressionSuite.scala | 121 ++++++++++-------- .../spark/streaming/TestSuiteBase.scala | 4 +- 3 files changed, 77 insertions(+), 55 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index d5c2e5ab54caa..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -91,6 +91,13 @@ junit-interface test + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 45e25eecf508e..28489410f8225 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -17,20 +17,19 @@ package org.apache.spark.mllib.regression -import java.io.File -import java.nio.charset.Charset - import scala.collection.mutable.ArrayBuffer -import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils +import org.apache.spark.mllib.util.LinearDataGenerator +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.TestSuiteBase + +class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { -class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { + // use longer wait time to ensure job completion + override def maxWaitTimeMillis = 20000 // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { @@ -49,35 +48,26 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data - test("streaming linear regression parameter accuracy") { + test("parameter accuracy") { - val testDir = Files.createTempDir() - val numBatches = 10 - val batchDuration = Milliseconds(1000) - val ssc = new StreamingContext(sc, batchDuration) - val data = ssc.textFileStream(testDir.toString).map(LabeledPoint.parse) + // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) .setStepSize(0.1) - .setNumIterations(50) + .setNumIterations(25) - model.trainOn(data) - - ssc.start() - - // write data to a file stream - for (i <- 0 until numBatches) { - val samples = LinearDataGenerator.generateLinearInput( - 0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) - val file = new File(testDir, i.toString) - Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) - Thread.sleep(batchDuration.milliseconds) + // generate sequence of simulated data + val numBatches = 10 + val input = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) } - ssc.stop(stopSparkContext=false) - - System.clearProperty("spark.driver.port") - Utils.deleteRecursively(testDir) + // apply model training to input stream + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) // check accuracy of final parameter estimates assertEqual(model.latestModel().intercept, 0.0, 0.1) @@ -91,39 +81,33 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } // Test that parameter estimates improve when learning Y = 10*X1 on streaming data - test("streaming linear regression parameter convergence") { + test("parameter convergence") { - val testDir = Files.createTempDir() - val batchDuration = Milliseconds(2000) - val ssc = new StreamingContext(sc, batchDuration) - val numBatches = 5 - val data = ssc.textFileStream(testDir.toString()).map(LabeledPoint.parse) + // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) .setStepSize(0.1) - .setNumIterations(50) - - model.trainOn(data) - - ssc.start() + .setNumIterations(25) - // write data to a file stream - val history = new ArrayBuffer[Double](numBatches) - for (i <- 0 until numBatches) { - val samples = LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) - val file = new File(testDir, i.toString) - Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) - Thread.sleep(batchDuration.milliseconds) - // wait an extra few seconds to make sure the update finishes before new data arrive - Thread.sleep(4000) - history.append(math.abs(model.latestModel().weights(0) - 10.0)) + // generate sequence of simulated data + val numBatches = 10 + val input = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) } - ssc.stop(stopSparkContext=false) + // create buffer to store intermediate fits + val history = new ArrayBuffer[Double](numBatches) - System.clearProperty("spark.driver.port") - Utils.deleteRecursively(testDir) + // apply model training to input stream, storing the intermediate results + // (we add a count to ensure the result is a DStream) + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - 10.0))) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + // compute change in error val deltas = history.drop(1).zip(history.dropRight(1)) // check error stability (it always either shrinks, or increases with small tol) assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) @@ -132,4 +116,33 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } + // Test predictions on a stream + test("predictions") { + + // create model initialized with true weights + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(10.0, 10.0)) + .setStepSize(0.1) + .setNumIterations(25) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), nPoints, 42 * (i + 1)) + } + + // apply model predictions to test stream + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + // collect the output as (true, estimated) tuples + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // compute the mean absolute error and check that it's always less than 0.1 + val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) + assert(errors.forall(x => x <= 0.1)) + + } + } 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 cc178fba12c9d..f095da9cb55d3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -242,7 +242,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput) // Get the output buffer - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]] + val outputStream = ssc.graph.getOutputStreams. + filter(_.isInstanceOf[TestOutputStreamWithPartitions[_]]). + head.asInstanceOf[TestOutputStreamWithPartitions[V]] val output = outputStream.output try { From c3952b092a2f7fea4798f4cb7abac300b9dc9c29 Mon Sep 17 00:00:00 2001 From: Vida Ha Date: Tue, 19 Aug 2014 13:35:05 -0700 Subject: [PATCH 044/192] SPARK-2333 - spark_ec2 script should allow option for existing security group - Uses the name tag to identify machines in a cluster. - Allows overriding the security group name so it doesn't need to coincide with the cluster name. - Outputs the request id's of up to 10 pending spot instance requests. Author: Vida Ha Closes #1899 from vidaha/vida/ec2-reuse-security-group and squashes the following commits: c80d5c3 [Vida Ha] wrap retries in a try catch block b2989d5 [Vida Ha] SPARK-2333: spark_ec2 script should allow option for existing security group (cherry picked from commit 94053a7b766788bb62e2dbbf352ccbcc75f71fc0) Signed-off-by: Josh Rosen --- docs/ec2-scripts.md | 14 +++++---- ec2/spark_ec2.py | 71 +++++++++++++++++++++++++++++++-------------- 2 files changed, 57 insertions(+), 28 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 156a727026790..f5ac6d894e1eb 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -12,14 +12,16 @@ on the [Amazon Web Services site](http://aws.amazon.com/). `spark-ec2` is designed to manage multiple named clusters. You can launch a new cluster (telling the script its size and giving it a name), -shutdown an existing cluster, or log into a cluster. Each cluster is -identified by placing its machines into EC2 security groups whose names -are derived from the name of the cluster. For example, a cluster named +shutdown an existing cluster, or log into a cluster. Each cluster +launches a set of instances, which are tagged with the cluster name, +and placed into EC2 security groups. If you don't specify a security +group, the `spark-ec2` script will create security groups based on the +cluster name you request. For example, a cluster named `test` will contain a master node in a security group called `test-master`, and a number of slave nodes in a security group called -`test-slaves`. The `spark-ec2` script will create these security groups -for you based on the cluster name you request. You can also use them to -identify machines belonging to each cluster in the Amazon EC2 Console. +`test-slaves`. You can also specify a security group prefix to be used +in place of the cluster name. Machines in a cluster can be identified +by looking for the "Name" tag of the instance in the Amazon EC2 Console. # Before You Start diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index fc6fb1db59424..a979891662fb7 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -124,7 +124,7 @@ def parse_args(): help="The SSH user you want to connect as (default: root)") parser.add_option( "--delete-groups", action="store_true", default=False, - help="When destroying a cluster, delete the security groups that were created") + help="When destroying a cluster, delete the security groups that were created.") parser.add_option( "--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") @@ -138,7 +138,9 @@ def parse_args(): parser.add_option( "--user-data", type="string", default="", help="Path to a user-data file (most AMI's interpret this as an initialization script)") - + parser.add_option( + "--security-group-prefix", type="string", default=None, + help="Use this prefix for the security group rather than the cluster name.") (opts, args) = parser.parse_args() if len(args) != 2: @@ -285,8 +287,12 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") + if opts.security_group_prefix is None: + master_group = get_or_make_group(conn, cluster_name + "-master") + slave_group = get_or_make_group(conn, cluster_name + "-slaves") + else: + master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") + slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) master_group.authorize(src_group=slave_group) @@ -310,12 +316,11 @@ def launch_cluster(conn, opts, cluster_name): slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') - # Check if instances are already running in our groups + # Check if instances are already running with the cluster name existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print >> stderr, ("ERROR: There are already instances for name: %s " % cluster_name) sys.exit(1) # Figure out Spark AMI @@ -371,9 +376,13 @@ def launch_cluster(conn, opts, cluster_name): for r in reqs: id_to_req[r.id] = r active_instance_ids = [] + outstanding_request_ids = [] for i in my_req_ids: - if i in id_to_req and id_to_req[i].state == "active": - active_instance_ids.append(id_to_req[i].instance_id) + if i in id_to_req: + if id_to_req[i].state == "active": + active_instance_ids.append(id_to_req[i].instance_id) + else: + outstanding_request_ids.append(i) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves reservations = conn.get_all_instances(active_instance_ids) @@ -382,8 +391,8 @@ def launch_cluster(conn, opts, cluster_name): slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print "%d of %d slaves granted, waiting longer for request ids including %s" % ( + len(active_instance_ids), opts.slaves, outstanding_request_ids[0:10]) except: print "Canceling spot instance requests" conn.cancel_spot_instance_requests(my_req_ids) @@ -440,14 +449,29 @@ def launch_cluster(conn, opts, cluster_name): print "Launched master in %s, regid = %s" % (zone, master_res.id) # Give the instances descriptive names + # TODO: Add retry logic for tagging with name since it's used to identify a cluster. for master in master_nodes: - master.add_tag( - key='Name', - value='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) + name = '{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id) + for i in range(0, 5): + try: + master.add_tag(key='Name', value=name) + except: + print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) + if (i == 5): + raise "Error - failed max attempts to add name tag" + time.sleep(5) + + for slave in slave_nodes: - slave.add_tag( - key='Name', - value='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) + name = '{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id) + for i in range(0, 5): + try: + slave.add_tag(key='Name', value=name) + except: + print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) + if (i == 5): + raise "Error - failed max attempts to add name tag" + time.sleep(5) # Return all the instances return (master_nodes, slave_nodes) @@ -463,10 +487,10 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for inst in active: - group_names = [g.name for g in inst.groups] - if group_names == [cluster_name + "-master"]: + name = inst.tags.get(u'Name', "") + if name.startswith(cluster_name + "-master"): master_nodes.append(inst) - elif group_names == [cluster_name + "-slaves"]: + elif name.startswith(cluster_name + "-slave"): slave_nodes.append(inst) if any((master_nodes, slave_nodes)): print ("Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))) @@ -474,7 +498,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in with name " + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) @@ -816,7 +840,10 @@ def real_main(): # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." - group_names = [cluster_name + "-master", cluster_name + "-slaves"] + if opts.security_group_prefix is None: + group_names = [cluster_name + "-master", cluster_name + "-slaves"] + else: + group_names = [opts.security_group_prefix + "-master", opts.security_group_prefix + "-slaves"] attempt = 1 while attempt <= 3: From f6b4ab83c073d84d1ca26f2ed1168fdbd1c928db Mon Sep 17 00:00:00 2001 From: hzw19900416 Date: Tue, 19 Aug 2014 14:04:49 -0700 Subject: [PATCH 045/192] Move a bracket in validateSettings of SparkConf Move a bracket in validateSettings of SparkConf Author: hzw19900416 Closes #2012 from hzw19900416/codereading and squashes the following commits: e717fb6 [hzw19900416] Move a bracket in validateSettings of SparkConf (cherry picked from commit 76eaeb4523ee01cabbea2d867daac48a277885a1) Signed-off-by: Josh Rosen --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 13f0bff7ee507..b4f321ec99e78 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -227,7 +227,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => if (javaOpts.contains("-Dspark")) { - val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts'). " + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." throw new Exception(msg) } From 3540d4b387568a4017fcd772233e4e10c1beb1b4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 19 Aug 2014 14:46:32 -0700 Subject: [PATCH 046/192] [SPARK-2790] [PySpark] fix zip with serializers which have different batch sizes. If two RDDs have different batch size in serializers, then it will try to re-serialize the one with smaller batch size, then call RDD.zip() in Spark. Author: Davies Liu Closes #1894 from davies/zip and squashes the following commits: c4652ea [Davies Liu] add more test cases 6d05fc8 [Davies Liu] Merge branch 'master' into zip 813b1e4 [Davies Liu] add more tests for failed cases a4aafda [Davies Liu] fix zip with serializers which have different batch sizes. (cherry picked from commit d7e80c2597d4a9cae2e0cb35a86f7889323f4cbb) Signed-off-by: Josh Rosen --- python/pyspark/rdd.py | 25 +++++++++++++++++++++++++ python/pyspark/serializers.py | 3 +++ python/pyspark/tests.py | 27 ++++++++++++++++++++++++++- 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 86cd89b245aea..140cbe05a43b0 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1687,6 +1687,31 @@ def zip(self, other): >>> x.zip(y).collect() [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)] """ + if self.getNumPartitions() != other.getNumPartitions(): + raise ValueError("Can only zip with RDD which has the same number of partitions") + + def get_batch_size(ser): + if isinstance(ser, BatchedSerializer): + return ser.batchSize + return 0 + + def batch_as(rdd, batchSize): + ser = rdd._jrdd_deserializer + if isinstance(ser, BatchedSerializer): + ser = ser.serializer + return rdd._reserialize(BatchedSerializer(ser, batchSize)) + + my_batch = get_batch_size(self._jrdd_deserializer) + other_batch = get_batch_size(other._jrdd_deserializer) + if my_batch != other_batch: + # use the greatest batchSize to batch the other one. + if my_batch > other_batch: + other = batch_as(other, my_batch) + else: + self = batch_as(self, other_batch) + + # There will be an Exception in JVM if there are different number + # of items in each partitions. pairRDD = self._jrdd.zip(other._jrdd) deserializer = PairDeserializer(self._jrdd_deserializer, other._jrdd_deserializer) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 74870c0edcf99..fc49aa42dbaf9 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -255,6 +255,9 @@ def __init__(self, key_ser, val_ser): def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): + if len(keys) != len(vals): + raise ValueError("Can not deserialize RDD with different number of items" + " in pair: (%d, %d)" % (len(keys), len(vals))) for pair in izip(keys, vals): yield pair diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 69d543d9d045d..51bfbb47e53c2 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -39,7 +39,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.serializers import read_int +from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger _have_scipy = False @@ -339,6 +339,31 @@ def test_large_broadcast(self): m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEquals(N, m) + def test_zip_with_different_serializers(self): + a = self.sc.parallelize(range(5)) + b = self.sc.parallelize(range(100, 105)) + self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)]) + a = a._reserialize(BatchedSerializer(PickleSerializer(), 2)) + b = b._reserialize(MarshalSerializer()) + self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)]) + + def test_zip_with_different_number_of_items(self): + a = self.sc.parallelize(range(5), 2) + # different number of partitions + b = self.sc.parallelize(range(100, 106), 3) + self.assertRaises(ValueError, lambda: a.zip(b)) + # different number of batched items in JVM + b = self.sc.parallelize(range(100, 104), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # different number of items in one pair + b = self.sc.parallelize(range(100, 106), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # same total number of items, but different distributions + a = self.sc.parallelize([2, 3], 2).flatMap(range) + b = self.sc.parallelize([3, 2], 2).flatMap(range) + self.assertEquals(a.count(), b.count()) + self.assertRaises(Exception, lambda: a.zip(b).count()) + class TestIO(PySparkTestCase): From d371c71cb19f62b1d2594f92f616abf09d9777a7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 16:06:48 -0700 Subject: [PATCH 047/192] [SPARK-3136][MLLIB] Create Java-friendly methods in RandomRDDs Though we don't use default argument for methods in RandomRDDs, it is still not easy for Java users to use because the output type is either `RDD[Double]` or `RDD[Vector]`. Java users should expect `JavaDoubleRDD` and `JavaRDD[Vector]`, respectively. We should create dedicated methods for Java users, and allow default arguments in Scala methods in RandomRDDs, to make life easier for both Java and Scala users. This PR also contains documentation for random data generation. brkyvz Author: Xiangrui Meng Closes #2041 from mengxr/stat-doc and squashes the following commits: fc5eedf [Xiangrui Meng] add missing comma ffde810 [Xiangrui Meng] address comments aef6d07 [Xiangrui Meng] add doc for random data generation b99d94b [Xiangrui Meng] add java-friendly methods to RandomRDDs (cherry picked from commit 825d4fe47b9c4d48de88622dd48dcf83beb8b80a) Signed-off-by: Xiangrui Meng --- docs/mllib-guide.md | 2 +- docs/mllib-stats.md | 74 ++- .../mllib/random/RandomDataGenerator.scala | 18 +- .../spark/mllib/random/RandomRDDs.scala | 476 +++++++----------- .../mllib/random/JavaRandomRDDsSuite.java | 134 +++++ python/pyspark/mllib/random.py | 20 +- 6 files changed, 418 insertions(+), 306 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 23d5a0c4607af..ca0a84a8c53fd 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -9,7 +9,7 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [Data types](mllib-basics.html) * [Basic statistics](mllib-stats.html) - * data generators + * random data generation * stratified sampling * summary statistics * hypothesis testing diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index ca9ef46c15186..f25dca746ba3a 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,7 +25,79 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` -## Data Generators +## Random data generation + +Random data generation is useful for randomized algorithms, prototyping, and performance testing. +MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +uniform, standard normal, or Poisson. + +
    +
    +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.random.RandomRDDs._ + +val sc: SparkContext = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +val u = normalRDD(sc, 1000000L, 10) +// Apply a transform to get a random double RDD following `N(1, 4)`. +val v = u.map(x => 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight java %} +import org.apache.spark.SparkContext; +import org.apache.spark.api.JavaDoubleRDD; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +JavaSparkContext jsc = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); +// Apply a transform to get a random double RDD following `N(1, 4)`. +JavaDoubleRDD v = u.map( + new Function() { + public Double call(Double x) { + return 1.0 + 2.0 * x; + } + }); +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight python %} +from pyspark.mllib.random import RandomRDDs + +sc = ... # SparkContext + +# Generate a random double RDD that contains 1 million i.i.d. values drawn from the +# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +# Apply a transform to get a random double RDD following `N(1, 4)`. +v = u.map(lambda x: 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    ## Stratified Sampling diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 9cab49f6ed1f0..28179fbc450c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -20,14 +20,14 @@ package org.apache.spark.mllib.random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} /** - * :: Experimental :: + * :: DeveloperApi :: * Trait for random data generators that generate i.i.d. data. */ -@Experimental +@DeveloperApi trait RandomDataGenerator[T] extends Pseudorandom with Serializable { /** @@ -43,10 +43,10 @@ trait RandomDataGenerator[T] extends Pseudorandom with Serializable { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from U[0.0, 1.0] */ -@Experimental +@DeveloperApi class UniformGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. @@ -62,10 +62,10 @@ class UniformGenerator extends RandomDataGenerator[Double] { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from the standard normal distribution. */ -@Experimental +@DeveloperApi class StandardNormalGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. @@ -81,12 +81,12 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from the Poisson distribution with the given mean. * * @param mean mean for the Poisson distribution. */ -@Experimental +@DeveloperApi class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { private var rng = new Poisson(mean, new DRand) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index 36270369526cd..c5f4b084321f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -20,9 +20,10 @@ package org.apache.spark.mllib.random import scala.reflect.ClassTag import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD, JavaSparkContext} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} +import org.apache.spark.mllib.rdd.{RandomRDD, RandomVectorRDD} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -34,335 +35,279 @@ import org.apache.spark.util.Utils object RandomRDDs { /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * Generates an RDD comprised of i.i.d. samples from the uniform distribution `U(0.0, 1.0)`. * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. + * To transform the distribution in the generated RDD from `U(0.0, 1.0)` to `U(a, b)`, use + * `RandomRDDs.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Double] comprised of i.i.d. samples ~ `U(0.0, 1.0)`. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + def uniformRDD( + sc: SparkContext, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { val uniform = new UniformGenerator() - randomRDD(sc, uniform, size, numPartitions, seed) + randomRDD(sc, uniform, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. - * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n, p).map(v => a + (b - a) * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * Java-friendly version of [[RandomRDDs#uniformRDD]]. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { - uniformRDD(sc, size, numPartitions, Utils.random.nextLong) + def uniformJavaRDD( + jsc: JavaSparkContext, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n).map(v => a + (b - a) * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * [[RandomRDDs#uniformJavaRDD]] with the default seed. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long): RDD[Double] = { - uniformRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. - * - * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and the default seed. */ - @Experimental - def normalRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { - val normal = new StandardNormalGenerator() - randomRDD(sc, normal, size, numPartitions, seed) + def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size)) } /** - * :: Experimental :: * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. * * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p).map(v => mean + sigma * v)`. + * `N(mean, sigma^2^)`, use `RandomRDDs.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). */ - @Experimental - def normalRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { - normalRDD(sc, size, numPartitions, Utils.random.nextLong) + def normalRDD( + sc: SparkContext, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val normal = new StandardNormalGenerator() + randomRDD(sc, normal, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n).map(v => mean + sigma * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + * Java-friendly version of [[RandomRDDs#normalRDD]]. */ - @Experimental - def normalRDD(sc: SparkContext, size: Long): RDD[Double] = { - normalRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + def normalJavaRDD( + jsc: JavaSparkContext, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#normalJavaRDD]] with the default seed. */ - @Experimental - def poissonRDD(sc: SparkContext, - mean: Double, - size: Long, - numPartitions: Int, - seed: Long): RDD[Double] = { - val poisson = new PoissonGenerator(mean) - randomRDD(sc, poisson, size, numPartitions, seed) + def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and the default seed. */ - @Experimental - def poissonRDD(sc: SparkContext, mean: Double, size: Long, numPartitions: Int): RDD[Double] = { - poissonRDD(sc, mean, size, numPartitions, Utils.random.nextLong) + def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size)) } /** - * :: Experimental :: * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. * @param mean Mean, or lambda, for the Poisson distribution. * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). */ - @Experimental - def poissonRDD(sc: SparkContext, mean: Double, size: Long): RDD[Double] = { - poissonRDD(sc, mean, size, sc.defaultParallelism, Utils.random.nextLong) + def poissonRDD( + sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val poisson = new PoissonGenerator(mean) + randomRDD(sc, poisson, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples produced by generator. + * Java-friendly version of [[RandomRDDs#poissonRDD]]. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, - generator: RandomDataGenerator[T], + def poissonJavaRDD( + jsc: JavaSparkContext, + mean: Double, size: Long, numPartitions: Int, - seed: Long): RDD[T] = { - new RandomRDD[T](sc, size, numPartitions, generator, seed) + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples produced by generator. + * [[RandomRDDs#poissonJavaRDD]] with the default seed. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, - generator: RandomDataGenerator[T], + def poissonJavaRDD( + jsc: JavaSparkContext, + mean: Double, size: Long, - numPartitions: Int): RDD[T] = { - randomRDD[T](sc, generator, size, numPartitions, Utils.random.nextLong) + numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * sc.defaultParallelism used for the number of partitions in the RDD. + * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and the default seed. + */ + def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size)) + } + + /** + * :: DeveloperApi :: + * Generates an RDD comprised of i.i.d. samples produced by the input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. + * @param generator RandomDataGenerator used to populate the RDD. * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples produced by generator. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, + @DeveloperApi + def randomRDD[T: ClassTag]( + sc: SparkContext, generator: RandomDataGenerator[T], - size: Long): RDD[T] = { - randomRDD[T](sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[T] = { + new RandomRDD[T](sc, size, numPartitionsOrDefault(sc, numPartitions), generator, seed) } // TODO Generate RDD[Vector] from multivariate distributions. /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. + * uniform distribution on `U(0.0, 1.0)`. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + * @return RDD[Vector] with vectors containing i.i.d samples ~ `U(0.0, 1.0)`. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, + def uniformVectorRDD( + sc: SparkContext, numRows: Long, numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { val uniform = new UniformGenerator() - randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + randomVectorRDD(sc, uniform, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + * Java-friendly version of [[RandomRDDs#uniformVectorRDD]]. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, + def uniformJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - uniformVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { - uniformVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + def uniformJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols, numPartitions).toJavaRDD() + } + + /** + * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions and the default seed. + */ + def uniformJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * standard normal distribution. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. + */ + def normalVectorRDD( + sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + val normal = new StandardNormalGenerator() + randomVectorRDD(sc, normal, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#normalVectorRDD]]. */ - @Experimental - def normalVectorRDD(sc: SparkContext, + def normalJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, numPartitions: Int, - seed: Long): RDD[Vector] = { - val uniform = new StandardNormalGenerator() - randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + seed: Long): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * standard normal distribution. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#normalJavaVectorRDD]] with the default seed. */ - @Experimental - def normalVectorRDD(sc: SparkContext, + def normalJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - normalVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols, numPartitions).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * standard normal distribution. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions and the default seed. */ - @Experimental - def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { - normalVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + def normalJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. * @@ -370,124 +315,85 @@ object RandomRDDs { * @param mean Mean, or lambda, for the Poisson distribution. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) + * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonVectorRDD( + sc: SparkContext, mean: Double, numRows: Long, numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { val poisson = new PoissonGenerator(mean) - randomVectorRDD(sc, poisson, numRows, numCols, numPartitions, seed) + randomVectorRDD(sc, poisson, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + * Java-friendly version of [[RandomRDDs#poissonVectorRDD]]. */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonJavaVectorRDD( + jsc: JavaSparkContext, mean: Double, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - poissonVectorRDD(sc, mean, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * Poisson distribution with the input mean. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed. */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonJavaVectorRDD( + jsc: JavaSparkContext, mean: Double, numRows: Long, - numCols: Int): RDD[Vector] = { - poissonVectorRDD(sc, mean, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions and the default seed. */ - @Experimental - def randomVectorRDD(sc: SparkContext, - generator: RandomDataGenerator[Double], + def poissonJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, numRows: Long, - numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { - new RandomVectorRDD(sc, numRows, numCols, numPartitions, generator, seed) + numCols: Int): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. + * input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. + * @param generator RandomDataGenerator used to populate the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. */ - @Experimental + @DeveloperApi def randomVectorRDD(sc: SparkContext, generator: RandomDataGenerator[Double], numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - randomVectorRDD(sc, generator, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + new RandomVectorRDD( + sc, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), generator, seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + * Returns `numPartitions` if it is positive, or `sc.defaultParallelism` otherwise. */ - @Experimental - def randomVectorRDD(sc: SparkContext, - generator: RandomDataGenerator[Double], - numRows: Long, - numCols: Int): RDD[Vector] = { - randomVectorRDD(sc, generator, numRows, numCols, - sc.defaultParallelism, Utils.random.nextLong) + private def numPartitionsOrDefault(sc: SparkContext, numPartitions: Int): Int = { + if (numPartitions > 0) numPartitions else sc.defaultMinPartitions } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java new file mode 100644 index 0000000000000..a725736ca1a58 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -0,0 +1,134 @@ +/* + * 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.mllib.random; + +import com.google.common.collect.Lists; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +public class JavaRandomRDDsSuite { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaRandomRDDsSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void testUniformRDD() { + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = uniformJavaRDD(sc, m); + JavaDoubleRDD rdd2 = uniformJavaRDD(sc, m, p); + JavaDoubleRDD rdd3 = uniformJavaRDD(sc, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + public void testNormalRDD() { + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = normalJavaRDD(sc, m); + JavaDoubleRDD rdd2 = normalJavaRDD(sc, m, p); + JavaDoubleRDD rdd3 = normalJavaRDD(sc, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + public void testPoissonRDD() { + double mean = 2.0; + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = poissonJavaRDD(sc, mean, m); + JavaDoubleRDD rdd2 = poissonJavaRDD(sc, mean, m, p); + JavaDoubleRDD rdd3 = poissonJavaRDD(sc, mean, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testUniformVectorRDD() { + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = uniformJavaVectorRDD(sc, m, n); + JavaRDD rdd2 = uniformJavaVectorRDD(sc, m, n, p); + JavaRDD rdd3 = uniformJavaVectorRDD(sc, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testNormalVectorRDD() { + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = normalJavaVectorRDD(sc, m, n); + JavaRDD rdd2 = normalJavaVectorRDD(sc, m, n, p); + JavaRDD rdd3 = normalJavaVectorRDD(sc, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testPoissonVectorRDD() { + double mean = 2.0; + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = poissonJavaVectorRDD(sc, mean, m, n); + JavaRDD rdd2 = poissonJavaVectorRDD(sc, mean, m, n, p); + JavaRDD rdd3 = poissonJavaVectorRDD(sc, mean, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } +} diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 3f3b19053d32e..4dc1a4a912421 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -35,10 +35,10 @@ class RandomRDDs: def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the - uniform distribution on [0.0, 1.0]. + uniform distribution U(0.0, 1.0). - To transform the distribution in the generated RDD from U[0.0, 1.0] - to U[a, b], use + To transform the distribution in the generated RDD from U(0.0, 1.0) + to U(a, b), use C{RandomRDDs.uniformRDD(sc, n, p, seed)\ .map(lambda v: a + (b - a) * v)} @@ -60,11 +60,11 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): @staticmethod def normalRDD(sc, size, numPartitions=None, seed=None): """ - Generates an RDD comprised of i.i.d samples from the standard normal + Generates an RDD comprised of i.i.d. samples from the standard normal distribution. To transform the distribution in the generated RDD from standard normal - to some other normal N(mean, sigma), use + to some other normal N(mean, sigma^2), use C{RandomRDDs.normal(sc, n, p, seed)\ .map(lambda v: mean + sigma * v)} @@ -84,7 +84,7 @@ def normalRDD(sc, size, numPartitions=None, seed=None): @staticmethod def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ - Generates an RDD comprised of i.i.d samples from the Poisson + Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. >>> mean = 100.0 @@ -105,8 +105,8 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): @staticmethod def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn - from the uniform distribution on [0.0 1.0]. + Generates an RDD comprised of vectors containing i.i.d. samples drawn + from the uniform distribution U(0.0, 1.0). >>> import numpy as np >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) @@ -125,7 +125,7 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): @staticmethod def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn + Generates an RDD comprised of vectors containing i.i.d. samples drawn from the standard normal distribution. >>> import numpy as np @@ -145,7 +145,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): @staticmethod def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn + Generates an RDD comprised of vectors containing i.i.d. samples drawn from the Poisson distribution with the input mean. >>> import numpy as np From 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Aug 2014 17:40:35 -0700 Subject: [PATCH 048/192] [SPARK-2468] Netty based block server / client module Previous pull request (#1907) was reverted. This brings it back. Still looking into the hang. Author: Reynold Xin Closes #1971 from rxin/netty1 and squashes the following commits: b0be96f [Reynold Xin] Added test to make sure outstandingRequests are cleaned after firing the events. 4c6d0ee [Reynold Xin] Pass callbacks cleanly. 603dce7 [Reynold Xin] Upgrade Netty to 4.0.23 to fix the DefaultFileRegion bug. 88be1d4 [Reynold Xin] Downgrade to 4.0.21 to work around a bug in writing DefaultFileRegion. 002626a [Reynold Xin] Remove netty-test-file.txt. db6e6e0 [Reynold Xin] Revert "Revert "[SPARK-2468] Netty based block server / client module"" (cherry picked from commit 8b9dc991018842e01f4b93870a2bc2c2cb9ea4ba) Signed-off-by: Reynold Xin --- .../spark/network/netty/FileClient.scala | 85 --------- .../network/netty/FileClientHandler.scala | 50 ------ .../spark/network/netty/FileHeader.scala | 71 -------- .../spark/network/netty/FileServer.scala | 91 ---------- .../network/netty/FileServerHandler.scala | 68 -------- .../spark/network/netty/NettyConfig.scala | 59 +++++++ .../spark/network/netty/ShuffleCopier.scala | 118 ------------- .../spark/network/netty/ShuffleSender.scala | 71 -------- .../BlockClientListener.scala} | 16 +- .../netty/client/BlockFetchingClient.scala | 132 ++++++++++++++ .../client/BlockFetchingClientFactory.scala | 99 +++++++++++ .../client/BlockFetchingClientHandler.scala | 103 +++++++++++ .../netty/client/LazyInitIterator.scala | 44 +++++ .../netty/client/ReferenceCountedBuffer.scala | 47 +++++ .../network/netty/server/BlockHeader.scala | 32 ++++ .../netty/server/BlockHeaderEncoder.scala | 47 +++++ .../network/netty/server/BlockServer.scala | 162 ++++++++++++++++++ .../BlockServerChannelInitializer.scala} | 22 ++- .../netty/server/BlockServerHandler.scala | 140 +++++++++++++++ .../spark/storage/BlockDataProvider.scala | 32 ++++ .../spark/storage/BlockFetcherIterator.scala | 137 +++++++-------- .../apache/spark/storage/BlockManager.scala | 49 +++++- .../storage/BlockNotFoundException.scala | 21 +++ .../spark/storage/DiskBlockManager.scala | 13 +- .../netty/ServerClientIntegrationSuite.scala | 161 +++++++++++++++++ .../BlockFetchingClientHandlerSuite.scala | 105 ++++++++++++ .../server/BlockHeaderEncoderSuite.scala | 64 +++++++ .../server/BlockServerHandlerSuite.scala | 107 ++++++++++++ pom.xml | 2 +- 29 files changed, 1484 insertions(+), 664 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala rename core/src/main/scala/org/apache/spark/network/netty/{FileClientChannelInitializer.scala => client/BlockClientListener.scala} (65%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala rename core/src/main/scala/org/apache/spark/network/netty/{FileServerChannelInitializer.scala => server/BlockServerChannelInitializer.scala} (58%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala deleted file mode 100644 index c6d35f73db545..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.TimeUnit - -import io.netty.bootstrap.Bootstrap -import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioSocketChannel - -import org.apache.spark.Logging - -class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { - - private var channel: Channel = _ - private var bootstrap: Bootstrap = _ - private var group: EventLoopGroup = _ - private val sendTimeout = 60 - - def init(): Unit = { - group = new OioEventLoopGroup - bootstrap = new Bootstrap - bootstrap.group(group) - .channel(classOf[OioSocketChannel]) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) - .handler(new FileClientChannelInitializer(handler)) - } - - def connect(host: String, port: Int) { - try { - channel = bootstrap.connect(host, port).sync().channel() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted while trying to connect", e) - close() - } - } - - def waitForClose(): Unit = { - try { - channel.closeFuture.sync() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted", e) - } - } - - def sendRequest(file: String): Unit = { - try { - val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) - if (!bSent) { - throw new RuntimeException("Failed to send") - } - } catch { - case e: InterruptedException => - logError("Error", e) - } - } - - def close(): Unit = { - if (group != null) { - group.shutdownGracefully() - group = null - bootstrap = null - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala deleted file mode 100644 index 017302ec7d33d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.storage.BlockId - - -abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { - - private var currentHeader: FileHeader = null - - @volatile - private var handlerCalled: Boolean = false - - def isComplete: Boolean = handlerCalled - - def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) - - def handleError(blockId: BlockId) - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { - currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) - } - if (in.readableBytes >= currentHeader.fileLen) { - handle(ctx, in, currentHeader) - handlerCalled = true - currentHeader = null - ctx.close() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala deleted file mode 100644 index 607e560ff277f..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, TestBlockId} - -private[spark] class FileHeader ( - val fileLen: Int, - val blockId: BlockId) extends Logging { - - lazy val buffer: ByteBuf = { - val buf = Unpooled.buffer() - buf.capacity(FileHeader.HEADER_SIZE) - buf.writeInt(fileLen) - buf.writeInt(blockId.name.length) - blockId.name.foreach((x: Char) => buf.writeByte(x)) - // padding the rest of header - if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { - buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) - } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") - } - buf - } - -} - -private[spark] object FileHeader { - - val HEADER_SIZE = 40 - - def getFileLenOffset = 0 - def getFileLenSize = Integer.SIZE/8 - - def create(buf: ByteBuf): FileHeader = { - val length = buf.readInt - val idLength = buf.readInt - val idBuilder = new StringBuilder(idLength) - for (i <- 1 to idLength) { - idBuilder += buf.readByte().asInstanceOf[Char] - } - val blockId = BlockId(idBuilder.toString()) - new FileHeader(length, blockId) - } - - def main(args:Array[String]) { - val header = new FileHeader(25, TestBlockId("my_block")) - val buf = header.buffer - val newHeader = FileHeader.create(buf) - System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala deleted file mode 100644 index dff77950659af..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.net.InetSocketAddress - -import io.netty.bootstrap.ServerBootstrap -import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioServerSocketChannel - -import org.apache.spark.Logging - -/** - * Server that accept the path of a file an echo back its content. - */ -class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { - - private val addr: InetSocketAddress = new InetSocketAddress(port) - private var bossGroup: EventLoopGroup = new OioEventLoopGroup - private var workerGroup: EventLoopGroup = new OioEventLoopGroup - - private var channelFuture: ChannelFuture = { - val bootstrap = new ServerBootstrap - bootstrap.group(bossGroup, workerGroup) - .channel(classOf[OioServerSocketChannel]) - .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) - .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) - .childHandler(new FileServerChannelInitializer(pResolver)) - bootstrap.bind(addr) - } - - try { - val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] - port = boundAddress.getPort - } catch { - case ie: InterruptedException => - port = 0 - } - - /** Start the file server asynchronously in a new thread. */ - def start(): Unit = { - val blockingThread: Thread = new Thread { - override def run(): Unit = { - try { - channelFuture.channel.closeFuture.sync - logInfo("FileServer exiting") - } catch { - case e: InterruptedException => - logError("File server start got interrupted", e) - } - // NOTE: bootstrap is shutdown in stop() - } - } - blockingThread.setDaemon(true) - blockingThread.start() - } - - def getPort: Int = port - - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bossGroup != null) { - bossGroup.shutdownGracefully() - bossGroup = null - } - if (workerGroup != null) { - workerGroup.shutdownGracefully() - workerGroup = null - } - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala deleted file mode 100644 index 96f60b2883ad9..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.FileInputStream - -import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, FileSegment} - - -class FileServerHandler(pResolver: PathResolver) - extends SimpleChannelInboundHandler[String] with Logging { - - override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { - val blockId: BlockId = BlockId(blockIdString) - val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) - if (fileSegment == null) { - return - } - val file = fileSegment.file - if (file.exists) { - if (!file.isFile) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - val length: Long = fileSegment.length - if (length > Integer.MAX_VALUE || length <= 0) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - ctx.write(new FileHeader(length.toInt, blockId).buffer) - try { - val channel = new FileInputStream(file).getChannel - ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) - } catch { - case e: Exception => - logError("Exception: ", e) - } - } else { - ctx.write(new FileHeader(0, blockId).buffer) - } - ctx.flush() - } - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError("Exception: ", cause) - ctx.close() - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala new file mode 100644 index 0000000000000..b5870152c5a64 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import org.apache.spark.SparkConf + +/** + * A central location that tracks all the settings we exposed to users. + */ +private[spark] +class NettyConfig(conf: SparkConf) { + + /** Port the server listens on. Default to a random port. */ + private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) + + /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ + private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase + + /** Connect timeout in secs. Default 60 secs. */ + private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 + + /** + * Percentage of the desired amount of time spent for I/O in the child event loops. + * Only applicable in nio and epoll. + */ + private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) + + /** Requested maximum length of the queue of incoming connections. */ + private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + private[netty] val receiveBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) + + /** Send buffer size (SO_SNDBUF). */ + private[netty] val sendBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala deleted file mode 100644 index e7b2855e1ec91..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.Executors - -import scala.collection.JavaConverters._ - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.util.CharsetUtil - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.BlockId - -private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { - - def getBlock(host: String, port: Int, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000) - val fc = new FileClient(handler, connectTimeout) - - try { - fc.init() - fc.connect(host, port) - fc.sendRequest(blockId.name) - fc.waitForClose() - fc.close() - } catch { - // Handle any socket-related exceptions in FileClient - case e: Exception => { - logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) - handler.handleError(blockId) - } - } - } - - def getBlock(cmId: ConnectionManagerId, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) - } - - def getBlocks(cmId: ConnectionManagerId, - blocks: Seq[(BlockId, Long)], - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - for ((blockId, size) <- blocks) { - getBlock(cmId, blockId, resultCollectCallback) - } - } -} - - -private[spark] object ShuffleCopier extends Logging { - - private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit) - extends FileClientHandler with Logging { - - override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)") - resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) - } - - override def handleError(blockId: BlockId) { - if (!isComplete) { - resultCollectCallBack(blockId, -1, null) - } - } - } - - def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) { - if (size != -1) { - logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") - } - } - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: ShuffleCopier ") - System.exit(1) - } - val host = args(0) - val port = args(1).toInt - val blockId = BlockId(args(2)) - val threads = if (args.length > 3) args(3).toInt else 10 - - val copiers = Executors.newFixedThreadPool(80) - val tasks = (for (i <- Range(0, threads)) yield { - Executors.callable(new Runnable() { - def run() { - val copier = new ShuffleCopier(new SparkConf) - copier.getBlock(host, port, blockId, echoResultCollectCallBack) - } - }) - }).asJava - copiers.invokeAll(tasks) - copiers.shutdown() - System.exit(0) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala deleted file mode 100644 index 95958e30f7eeb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.File - -import org.apache.spark.Logging -import org.apache.spark.util.Utils -import org.apache.spark.storage.{BlockId, FileSegment} - -private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { - - val server = new FileServer(pResolver, portIn) - server.start() - - def stop() { - server.stop() - } - - def port: Int = server.getPort -} - - -/** - * An application for testing the shuffle sender as a standalone program. - */ -private[spark] object ShuffleSender { - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println( - "Usage: ShuffleSender ") - System.exit(1) - } - - val port = args(0).toInt - val subDirsPerLocalDir = args(1).toInt - val localDirs = args.drop(2).map(new File(_)) - - val pResovler = new PathResolver { - override def getBlockLocation(blockId: BlockId): FileSegment = { - if (!blockId.isShuffle) { - throw new Exception("Block " + blockId + " is not a shuffle block") - } - // Figure out which local directory it hashes to, and which subdirectory in that - val hash = Utils.nonNegativeHash(blockId) - val dirId = hash % localDirs.length - val subDirId = (hash / localDirs.length) % subDirsPerLocalDir - val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) - val file = new File(subDir, blockId.name) - new FileSegment(file, 0, file.length()) - } - } - val sender = new ShuffleSender(port, pResovler) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala index f4261c13f70a8..e28219dd7745b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala @@ -15,17 +15,15 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.netty.client -import io.netty.channel.ChannelInitializer -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.string.StringEncoder +import java.util.EventListener -class FileClientChannelInitializer(handler: FileClientHandler) - extends ChannelInitializer[SocketChannel] { +trait BlockClientListener extends EventListener { + + def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit + + def onFetchFailure(blockId: String, errorMsg: String): Unit - def initChannel(channel: SocketChannel) { - channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) - } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala new file mode 100644 index 0000000000000..5aea7ba2f3673 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.util.concurrent.TimeoutException + +import io.netty.bootstrap.Bootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.socket.SocketChannel +import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.string.StringEncoder +import io.netty.util.CharsetUtil + +import org.apache.spark.Logging + +/** + * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. + * Use [[BlockFetchingClientFactory]] to instantiate this client. + * + * The constructor blocks until a connection is successfully established. + * + * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +@throws[TimeoutException] +private[spark] +class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) + extends Logging { + + private val handler = new BlockFetchingClientHandler + + /** Netty Bootstrap for creating the TCP connection. */ + private val bootstrap: Bootstrap = { + val b = new Bootstrap + b.group(factory.workerGroup) + .channel(factory.socketChannelClass) + // Use pooled buffers to reduce temporary buffer allocation + .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) + + b.handler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) + // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 + .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) + .addLast("handler", handler) + } + }) + b + } + + /** Netty ChannelFuture for the connection. */ + private val cf: ChannelFuture = bootstrap.connect(hostname, port) + if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { + throw new TimeoutException( + s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") + } + + /** + * Ask the remote server for a sequence of blocks, and execute the callback. + * + * Note that this is asynchronous and returns immediately. Upstream caller should throttle the + * rate of fetching; otherwise we could run out of memory. + * + * @param blockIds sequence of block ids to fetch. + * @param listener callback to fire on fetch success / failure. + */ + def fetchBlocks(blockIds: Seq[String], listener: BlockClientListener): Unit = { + // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. + // It's also best to limit the number of "flush" calls since it requires system calls. + // Let's concatenate the string and then call writeAndFlush once. + // This is also why this implementation might be more efficient than multiple, separate + // fetch block calls. + var startTime: Long = 0 + logTrace { + startTime = System.nanoTime + s"Sending request $blockIds to $hostname:$port" + } + + blockIds.foreach { blockId => + handler.addRequest(blockId, listener) + } + + val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") + writeFuture.addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture): Unit = { + if (future.isSuccess) { + logTrace { + val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 + s"Sending request $blockIds to $hostname:$port took $timeTaken ms" + } + } else { + // Fail all blocks. + val errorMsg = + s"Failed to send request $blockIds to $hostname:$port: ${future.cause.getMessage}" + logError(errorMsg, future.cause) + blockIds.foreach { blockId => + listener.onFetchFailure(blockId, errorMsg) + handler.removeRequest(blockId) + } + } + } + }) + } + + def waitForClose(): Unit = { + cf.channel().closeFuture().sync() + } + + def close(): Unit = cf.channel().close() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala new file mode 100644 index 0000000000000..2b28402c52b49 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.nio.NioSocketChannel +import io.netty.channel.socket.oio.OioSocketChannel +import io.netty.channel.{EventLoopGroup, Channel} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.util.Utils + +/** + * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses + * the worker thread pool for Netty. + * + * Concurrency: createClient is safe to be called from multiple threads concurrently. + */ +private[spark] +class BlockFetchingClientFactory(val conf: NettyConfig) { + + def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) + + /** A thread factory so the threads are named (for debugging). */ + val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") + + /** The following two are instantiated by the [[init]] method, depending ioMode. */ + var socketChannelClass: Class[_ <: Channel] = _ + var workerGroup: EventLoopGroup = _ + + init() + + /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ + private def init(): Unit = { + def initOio(): Unit = { + socketChannelClass = classOf[OioSocketChannel] + workerGroup = new OioEventLoopGroup(0, threadFactory) + } + def initNio(): Unit = { + socketChannelClass = classOf[NioSocketChannel] + workerGroup = new NioEventLoopGroup(0, threadFactory) + } + def initEpoll(): Unit = { + socketChannelClass = classOf[EpollSocketChannel] + workerGroup = new EpollEventLoopGroup(0, threadFactory) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + } + + /** + * Create a new BlockFetchingClient connecting to the given remote host / port. + * + * This blocks until a connection is successfully established. + * + * Concurrency: This method is safe to call from multiple threads. + */ + def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { + new BlockFetchingClient(this, remoteHost, remotePort) + } + + def stop(): Unit = { + if (workerGroup != null) { + workerGroup.shutdownGracefully() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala new file mode 100644 index 0000000000000..83265b164299d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging + + +/** + * Handler that processes server responses. It uses the protocol documented in + * [[org.apache.spark.network.netty.server.BlockServer]]. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +private[client] +class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { + + /** Tracks the list of outstanding requests and their listeners on success/failure. */ + private val outstandingRequests = java.util.Collections.synchronizedMap { + new java.util.HashMap[String, BlockClientListener] + } + + def addRequest(blockId: String, listener: BlockClientListener): Unit = { + outstandingRequests.put(blockId, listener) + } + + def removeRequest(blockId: String): Unit = { + outstandingRequests.remove(blockId) + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + val errorMsg = s"Exception in connection from ${ctx.channel.remoteAddress}: ${cause.getMessage}" + logError(errorMsg, cause) + + // Fire the failure callback for all outstanding blocks + outstandingRequests.synchronized { + val iter = outstandingRequests.entrySet().iterator() + while (iter.hasNext) { + val entry = iter.next() + entry.getValue.onFetchFailure(entry.getKey, errorMsg) + } + outstandingRequests.clear() + } + + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + val totalLen = in.readInt() + val blockIdLen = in.readInt() + val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) + in.readBytes(blockIdBytes) + val blockId = new String(blockIdBytes) + val blockSize = totalLen - math.abs(blockIdLen) - 4 + + def server = ctx.channel.remoteAddress.toString + + // blockIdLen is negative when it is an error message. + if (blockIdLen < 0) { + val errorMessageBytes = new Array[Byte](blockSize) + in.readBytes(errorMessageBytes) + val errorMsg = new String(errorMessageBytes) + logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") + + val listener = outstandingRequests.get(blockId) + if (listener == null) { + // Ignore callback + logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") + } else { + outstandingRequests.remove(blockId) + listener.onFetchFailure(blockId, errorMsg) + } + } else { + logTrace(s"Received block $blockId ($blockSize B) from $server") + + val listener = outstandingRequests.get(blockId) + if (listener == null) { + // Ignore callback + logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") + } else { + outstandingRequests.remove(blockId) + listener.onFetchSuccess(blockId, new ReferenceCountedBuffer(in)) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala new file mode 100644 index 0000000000000..9740ee64d1f2d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +/** + * A simple iterator that lazily initializes the underlying iterator. + * + * The use case is that sometimes we might have many iterators open at the same time, and each of + * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). + * This could lead to too many buffers open. If this iterator is used, we lazily initialize those + * buffers. + */ +private[spark] +class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { + + lazy val proxy = createIterator + + override def hasNext: Boolean = { + val gotNext = proxy.hasNext + if (!gotNext) { + close() + } + gotNext + } + + override def next(): Any = proxy.next() + + def close(): Unit = Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala new file mode 100644 index 0000000000000..ea1abf5eccc26 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.io.InputStream +import java.nio.ByteBuffer + +import io.netty.buffer.{ByteBuf, ByteBufInputStream} + + +/** + * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. + * This is a Scala value class. + * + * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of + * reference by the retain method and release method. + */ +private[spark] +class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { + + /** Return the nio ByteBuffer view of the underlying buffer. */ + def byteBuffer(): ByteBuffer = underlying.nioBuffer + + /** Creates a new input stream that starts from the current position of the buffer. */ + def inputStream(): InputStream = new ByteBufInputStream(underlying) + + /** Increment the reference counter by one. */ + def retain(): Unit = underlying.retain() + + /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ + def release(): Unit = underlying.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala new file mode 100644 index 0000000000000..162e9cc6828d4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +/** + * Header describing a block. This is used only in the server pipeline. + * + * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. + * + * @param blockSize length of the block content, excluding the length itself. + * If positive, this is the header for a block (not part of the header). + * If negative, this is the header and content for an error message. + * @param blockId block id + * @param error some error message from reading the block + */ +private[server] +class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala new file mode 100644 index 0000000000000..8e4dda4ef8595 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.handler.codec.MessageToByteEncoder + +/** + * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. + */ +private[server] +class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { + override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { + // message = message length (4 bytes) + block id length (4 bytes) + block id + block data + // message length = block id length (4 bytes) + size of block id + size of block data + val blockIdBytes = msg.blockId.getBytes + msg.error match { + case Some(errorMsg) => + val errorBytes = errorMsg.getBytes + out.writeInt(4 + blockIdBytes.length + errorBytes.size) + out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors + out.writeBytes(blockIdBytes) // next is blockId itself + out.writeBytes(errorBytes) // error message + case None => + out.writeInt(4 + blockIdBytes.length + msg.blockSize) + out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length + out.writeBytes(blockIdBytes) // next is blockId itself + // msg of size blockSize will be written by ServerHandler + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala new file mode 100644 index 0000000000000..7b2f9a8d4dfd0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.channel.socket.oio.OioServerSocketChannel +import io.netty.handler.codec.LineBasedFrameDecoder +import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.storage.BlockDataProvider +import org.apache.spark.util.Utils + + +/** + * Server for serving Spark data blocks. + * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. + * + * Protocol for requesting blocks (client to server): + * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" + * + * Protocol for sending blocks (server to client): + * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. + * + * frame-length should not include the length of itself. + * If block-id-length is negative, then this is an error message rather than block-data. The real + * length is the absolute value of the frame-length. + * + */ +private[spark] +class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { + + def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { + this(new NettyConfig(sparkConf), dataProvider) + } + + def port: Int = _port + + def hostName: String = _hostName + + private var _port: Int = conf.serverPort + private var _hostName: String = "" + private var bootstrap: ServerBootstrap = _ + private var channelFuture: ChannelFuture = _ + + init() + + /** Initialize the server. */ + private def init(): Unit = { + bootstrap = new ServerBootstrap + val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") + val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") + + // Use only one thread to accept connections, and 2 * num_cores for worker. + def initNio(): Unit = { + val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) + } + def initOio(): Unit = { + val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) + bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) + } + def initEpoll(): Unit = { + val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) + val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + + // Use pooled buffers to reduce temporary buffer allocation + bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + + // Various (advanced) user-configured settings. + conf.backLog.foreach { backLog => + bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) + } + conf.receiveBuf.foreach { receiveBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) + } + conf.sendBuf.foreach { sendBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) + } + + bootstrap.childHandler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(_port)) + channelFuture.sync() + + val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] + _port = addr.getPort + _hostName = addr.getHostName + } + + /** Shutdown the server. */ + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala similarity index 58% rename from core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala index aaa2f913d0269..cc70bd0c5c477 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala @@ -15,20 +15,26 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.netty.server import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} +import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil +import org.apache.spark.storage.BlockDataProvider -class FileServerChannelInitializer(pResolver: PathResolver) + +/** Channel initializer that sets up the pipeline for the BlockServer. */ +private[netty] +class BlockServerChannelInitializer(dataProvider: BlockDataProvider) extends ChannelInitializer[SocketChannel] { - override def initChannel(channel: SocketChannel): Unit = { - channel.pipeline - .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) - .addLast("stringDecoder", new StringDecoder) - .addLast("handler", new FileServerHandler(pResolver)) + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala new file mode 100644 index 0000000000000..40dd5e5d1a2ac --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.FileInputStream +import java.nio.ByteBuffer +import java.nio.channels.FileChannel + +import io.netty.buffer.Unpooled +import io.netty.channel._ + +import org.apache.spark.Logging +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * A handler that processes requests from clients and writes block data back. + * + * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first + * so channelRead0 is called once per line (i.e. per block id). + */ +private[server] +class BlockServerHandler(dataProvider: BlockDataProvider) + extends SimpleChannelInboundHandler[String] with Logging { + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { + def client = ctx.channel.remoteAddress.toString + + // A helper function to send error message back to the client. + def respondWithError(error: String): Unit = { + ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( + new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (!future.isSuccess) { + // TODO: Maybe log the success case as well. + logError(s"Error sending error back to $client", future.cause) + ctx.close() + } + } + } + ) + } + + def writeFileSegment(segment: FileSegment): Unit = { + // Send error message back if the block is too large. Even though we are capable of sending + // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. + // Once we fixed the receiving end to be able to process large blocks, this should be removed. + // Also make sure we update BlockHeaderEncoder to support length > 2G. + + // See [[BlockHeaderEncoder]] for the way length is encoded. + if (segment.length + blockId.length + 4 > Int.MaxValue) { + respondWithError(s"Block $blockId size ($segment.length) greater than 2G") + return + } + + var fileChannel: FileChannel = null + try { + fileChannel = new FileInputStream(segment.file).getChannel + } catch { + case e: Exception => + logError( + s"Error opening channel for $blockId in ${segment.file} for request from $client", e) + respondWithError(e.getMessage) + } + + // Found the block. Send it back. + if (fileChannel != null) { + // Write the header and block data. In the case of failures, the listener on the block data + // write should close the connection. + ctx.write(new BlockHeader(segment.length.toInt, blockId)) + + val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) + ctx.writeAndFlush(region).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${segment.length} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + } + + def writeByteBuffer(buf: ByteBuffer): Unit = { + ctx.write(new BlockHeader(buf.remaining, blockId)) + ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + + logTrace(s"Received request from $client to fetch block $blockId") + + var blockData: Either[FileSegment, ByteBuffer] = null + + // First make sure we can find the block. If not, send error back to the user. + try { + blockData = dataProvider.getBlockData(blockId) + } catch { + case e: Exception => + logError(s"Error opening block $blockId for request from $client", e) + respondWithError(e.getMessage) + return + } + + blockData match { + case Left(segment) => writeFileSegment(segment) + case Right(buf) => writeByteBuffer(buf) + } + + } // end of channelRead0 +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala new file mode 100644 index 0000000000000..5b6d086630834 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.ByteBuffer + + +/** + * An interface for providing data for blocks. + * + * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. + * + * Aside from unit tests, [[BlockManager]] is the main class that implements this. + */ +private[spark] trait BlockDataProvider { + def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] +} 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 5f44f5f3197fd..ca60ec78b62ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -18,19 +18,17 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue +import org.apache.spark.network.netty.client.{BlockClientListener, LazyInitIterator, ReferenceCountedBuffer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue import scala.util.{Failure, Success} -import io.netty.buffer.ByteBuf - import org.apache.spark.{Logging, SparkException} import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -54,18 +52,28 @@ trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] wi private[storage] object BlockFetcherIterator { - // A request to fetch one or more blocks, complete with their sizes + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { def failed: Boolean = size == -1 } + // TODO: Refactor this whole thing to make code more reusable. class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], @@ -95,10 +103,10 @@ object BlockFetcherIterator { // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - private val fetchRequests = new Queue[FetchRequest] + protected val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - private var bytesInFlight = 0L + protected var bytesInFlight = 0L protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( @@ -262,77 +270,58 @@ object BlockFetcherIterator { readMetrics: ShuffleReadMetrics) extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { - import blockManager._ + override protected def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + + // This could throw a TimeoutException. In that case we will just retry the task. + val client = blockManager.nettyBlockClientFactory.createClient( + cmId.host, req.address.nettyPort) + val blocks = req.blocks.map(_._1.toString) + + client.fetchBlocks( + blocks, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = { + logError(s"Could not get block(s) from $cmId with error: $errorMsg") + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } - private def startCopiers(numCopiers: Int): List[_ <: Thread] = { - (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { - sendRequest(fetchRequestsSync.take()) + override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { + // Increment the reference count so the buffer won't be recycled. + // TODO: This could result in memory leaks when the task is stopped due to exception + // before the iterator is exhausted. + data.retain() + val buf = data.byteBuffer() + val blockSize = buf.remaining() + val bid = BlockId(blockId) + + // TODO: remove code duplication between here and BlockManager.dataDeserialization. + results.put(new FetchResult(bid, sizeMap(bid), () => { + def createIterator: Iterator[Any] = { + val stream = blockManager.wrapForCompression(bid, data.inputStream()) + serializer.newInstance().deserializeStream(stream).asIterator } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - // case _ => throw new SparkException("Exception Throw in Shuffle Copier") + new LazyInitIterator(createIterator) { + // Release the buffer when we are done traversing it. + override def close(): Unit = data.release() + } + })) + + readMetrics.synchronized { + readMetrics.remoteBytesRead += blockSize + readMetrics.remoteBlocksFetched += 1 } + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } - copier.start - copier - }).toList - } - - // keep this to interrupt the threads when necessary - private def stopCopiers() { - for (copier <- copiers) { - copier.interrupt() - } - } - - override protected def sendRequest(req: FetchRequest) { - - def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) { - val fetchResult = new FetchResult(blockId, blockSize, - () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) - results.put(fetchResult) - } - - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier(blockManager.conf) - cpier.getBlocks(cmId, req.blocks, putResult) - logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) - } - - private var copiers: List[_ <: Thread] = null - - override def initialize() { - // Split Local Remote Blocks and set numBlocksToFetch - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - for (request <- Utils.randomize(remoteRequests)) { - fetchRequestsSync.put(request) - } - - copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + - Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - // If all the results has been retrieved, copiers will exit automatically - (result.blockId, if (result.failed) None else Some(result.deserialize())) + ) } } // End of NettyBlockFetcherIterator 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 e4c3d58905e7f..c0491fb55e3a4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -25,17 +25,20 @@ import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Cancellable, Props} +import akka.actor.{ActorSystem, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ +import org.apache.spark.network.netty.client.BlockFetchingClientFactory +import org.apache.spark.network.netty.server.BlockServer import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ + private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -60,7 +63,7 @@ private[spark] class BlockManager( securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager) - extends Logging { + extends BlockDataProvider with Logging { private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) @@ -88,13 +91,25 @@ private[spark] class BlockManager( new TachyonStore(this, tachyonBlockManager) } + private val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) + // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private val nettyPort: Int = { - val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) - val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0) - if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 + private[storage] val nettyBlockClientFactory: BlockFetchingClientFactory = { + if (useNetty) new BlockFetchingClientFactory(conf) else null } + private val nettyBlockServer: BlockServer = { + if (useNetty) { + val server = new BlockServer(conf, this) + logInfo(s"Created NettyBlockServer binding to port: ${server.port}") + server + } else { + null + } + } + + private val nettyPort: Int = if (useNetty) nettyBlockServer.port else 0 + val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) @@ -219,6 +234,20 @@ private[spark] class BlockManager( } } + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + val bid = BlockId(blockId) + if (bid.isShuffle) { + Left(diskBlockManager.getBlockLocation(bid)) + } else { + val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + if (blockBytesOpt.isDefined) { + Right(blockBytesOpt.get) + } else { + throw new BlockNotFoundException(blockId) + } + } + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -1064,6 +1093,14 @@ private[spark] class BlockManager( connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() + + if (nettyBlockClientFactory != null) { + nettyBlockClientFactory.stop() + } + if (nettyBlockServer != null) { + nettyBlockServer.stop() + } + actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala new file mode 100644 index 0000000000000..9ef453605f4f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + + +class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 4d66ccea211fa..f3da816389581 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -23,7 +23,7 @@ import java.util.{Date, Random, UUID} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.{PathResolver, ShuffleSender} +import org.apache.spark.network.netty.PathResolver import org.apache.spark.util.Utils import org.apache.spark.shuffle.sort.SortShuffleManager @@ -52,7 +52,6 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - private var shuffleSender : ShuffleSender = null addShutdownHook() @@ -186,15 +185,5 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, } } } - - if (shuffleSender != null) { - shuffleSender.stop() - } - } - - private[storage] def startShuffleBlockSender(port: Int): Int = { - shuffleSender = new ShuffleSender(port, this) - logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") - shuffleSender.port } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala new file mode 100644 index 0000000000000..02d0ffc86f58f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.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.network.netty + +import java.io.{RandomAccessFile, File} +import java.nio.ByteBuffer +import java.util.{Collections, HashSet} +import java.util.concurrent.{TimeUnit, Semaphore} + +import scala.collection.JavaConversions._ + +import io.netty.buffer.{ByteBufUtil, Unpooled} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.client.{BlockClientListener, ReferenceCountedBuffer, BlockFetchingClientFactory} +import org.apache.spark.network.netty.server.BlockServer +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * Test suite that makes sure the server and the client implementations share the same protocol. + */ +class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { + + val bufSize = 100000 + var buf: ByteBuffer = _ + var testFile: File = _ + var server: BlockServer = _ + var clientFactory: BlockFetchingClientFactory = _ + + val bufferBlockId = "buffer_block" + val fileBlockId = "file_block" + + val fileContent = new Array[Byte](1024) + scala.util.Random.nextBytes(fileContent) + + override def beforeAll() = { + buf = ByteBuffer.allocate(bufSize) + for (i <- 1 to bufSize) { + buf.put(i.toByte) + } + buf.flip() + + testFile = File.createTempFile("netty-test-file", "txt") + val fp = new RandomAccessFile(testFile, "rw") + fp.write(fileContent) + fp.close() + + server = new BlockServer(new SparkConf, new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + if (blockId == bufferBlockId) { + Right(buf) + } else if (blockId == fileBlockId) { + Left(new FileSegment(testFile, 10, testFile.length - 25)) + } else { + throw new Exception("Unknown block id " + blockId) + } + } + }) + + clientFactory = new BlockFetchingClientFactory(new SparkConf) + } + + override def afterAll() = { + server.stop() + clientFactory.stop() + } + + /** A ByteBuf for buffer_block */ + lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) + + /** A ByteBuf for file_block */ + lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) + + def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = + { + val client = clientFactory.createClient(server.hostName, server.port) + val sem = new Semaphore(0) + val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) + val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) + val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) + + client.fetchBlocks( + blockIds, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = { + errorBlockIds.add(blockId) + sem.release() + } + + override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { + receivedBlockIds.add(blockId) + data.retain() + receivedBuffers.add(data) + sem.release() + } + } + ) + if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server") + } + client.close() + (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) + } + + test("fetch a ByteBuffer block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a FileSegment block via zero-copy send") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) + assert(blockIds === Set(fileBlockId)) + assert(buffers.map(_.underlying) === Set(fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) + assert(blockIds.isEmpty) + assert(buffers.isEmpty) + assert(failBlockIds === Set("random-block")) + } + + test("fetch both ByteBuffer block and FileSegment block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) + assert(blockIds === Set(bufferBlockId, fileBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch both ByteBuffer block and a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds === Set("random-block")) + buffers.foreach(_.release()) + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala new file mode 100644 index 0000000000000..903ab09ae4322 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.nio.ByteBuffer + +import io.netty.buffer.Unpooled +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.{PrivateMethodTester, FunSuite} + + +class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester { + + test("handling block data (successful fetch)") { + val blockId = "test_block" + val blockData = "blahblahblahblahblah" + val totalLength = 4 + blockId.length + blockData.length + + var parsedBlockId: String = "" + var parsedBlockData: String = "" + val handler = new BlockFetchingClientHandler + handler.addRequest(blockId, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = ??? + override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer): Unit = { + parsedBlockId = bid + val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) + refCntBuf.byteBuffer().get(bytes) + parsedBlockData = new String(bytes) + } + } + ) + + val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) + assert(handler.invokePrivate(outstandingRequests()).size === 1) + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(blockId.length) + buf.put(blockId.getBytes) + buf.put(blockData.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedBlockData === blockData) + + assert(handler.invokePrivate(outstandingRequests()).size === 0) + + channel.close() + } + + test("handling error message (failed fetch)") { + val blockId = "test_block" + val errorMsg = "error erro5r error err4or error3 error6 error erro1r" + val totalLength = 4 + blockId.length + errorMsg.length + + var parsedBlockId: String = "" + var parsedErrorMsg: String = "" + val handler = new BlockFetchingClientHandler + handler.addRequest(blockId, new BlockClientListener { + override def onFetchFailure(bid: String, msg: String) ={ + parsedBlockId = bid + parsedErrorMsg = msg + } + override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer) = ??? + }) + + val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) + assert(handler.invokePrivate(outstandingRequests()).size === 1) + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(-blockId.length) + buf.put(blockId.getBytes) + buf.put(errorMsg.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedErrorMsg === errorMsg) + + assert(handler.invokePrivate(outstandingRequests()).size === 0) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala new file mode 100644 index 0000000000000..3ee281cb1350b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + + +class BlockHeaderEncoderSuite extends FunSuite { + + test("encode normal block data") { + val blockId = "test_block" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, None)) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + 17) + assert(out.readInt() === blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + assert(out.readableBytes() === 0) + + channel.close() + } + + test("encode error message") { + val blockId = "error_block" + val errorMsg = "error encountered" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + errorMsg.length) + assert(out.readInt() === -blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + + val errorMsgBytes = new Array[Byte](errorMsg.length) + out.readBytes(errorMsgBytes) + assert(new String(errorMsgBytes) === errorMsg) + assert(out.readableBytes() === 0) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala new file mode 100644 index 0000000000000..3239c710f1639 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.{RandomAccessFile, File} +import java.nio.ByteBuffer + +import io.netty.buffer.{Unpooled, ByteBuf} +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + +import org.apache.spark.storage.{BlockDataProvider, FileSegment} + + +class BlockServerHandlerSuite extends FunSuite { + + test("ByteBuffer block") { + val expectedBlockId = "test_bytebuffer_block" + val buf = ByteBuffer.allocate(10000) + for (i <- 1 to 10000) { + buf.put(i.toByte) + } + buf.flip() + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[ByteBuf] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === buf.remaining) + assert(out1.error === None) + + assert(out2.equals(Unpooled.wrappedBuffer(buf))) + + channel.close() + } + + test("FileSegment block via zero-copy") { + val expectedBlockId = "test_file_block" + + // Create random file data + val fileContent = new Array[Byte](1024) + scala.util.Random.nextBytes(fileContent) + val testFile = File.createTempFile("netty-test-file", "txt") + val fp = new RandomAccessFile(testFile, "rw") + fp.write(fileContent) + fp.close() + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + Left(new FileSegment(testFile, 15, testFile.length - 25)) + } + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === testFile.length - 25) + assert(out1.error === None) + + assert(out2.count === testFile.length - 25) + assert(out2.position === 15) + } + + test("pipeline exception propagation") { + val blockServerHandler = new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? + }) + val exceptionHandler = new SimpleChannelInboundHandler[String]() { + override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { + throw new Exception("this is an error") + } + } + + val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) + assert(channel.isOpen) + channel.writeInbound("a message to trigger the error") + assert(!channel.isOpen) + } +} diff --git a/pom.xml b/pom.xml index 9e5217e294681..8c4c4af0eda8e 100644 --- a/pom.xml +++ b/pom.xml @@ -419,7 +419,7 @@ io.netty netty-all - 4.0.17.Final + 4.0.23.Final org.apache.derby From 023ed7c0fe9b491dd8d699532260cc2d1c258ebb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 17:41:37 -0700 Subject: [PATCH 049/192] [MLLIB] minor update to word2vec very minor update Ishiihara Author: Xiangrui Meng Closes #2043 from mengxr/minor-w2v and squashes the following commits: be649fd [Xiangrui Meng] remove map because we only need append eccefcc [Xiangrui Meng] minor updates to word2vec (cherry picked from commit 1870dbaa5591883e61b2173d064c1a67e871b0f5) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/feature/Word2Vec.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 1dcaa2cd2e630..c3375ed44fd99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -30,11 +30,9 @@ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** * Entry in vocabulary @@ -285,9 +283,9 @@ class Word2Vec extends Serializable with Logging { val newSentences = sentences.repartition(numPartitions).cache() val initRandom = new XORShiftRandom(seed) - var syn0Global = + val syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) - var syn1Global = new Array[Float](vocabSize * vectorSize) + val syn1Global = new Array[Float](vocabSize * vectorSize) var alpha = startingAlpha for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => @@ -349,21 +347,21 @@ class Word2Vec extends Serializable with Logging { } val syn0Local = model._1 val syn1Local = model._2 - val synOut = new PrimitiveKeyOpenHashMap[Int, Array[Float]](vocabSize * 2) + val synOut = mutable.ListBuffer.empty[(Int, Array[Float])] var index = 0 while(index < vocabSize) { if (syn0Modify(index) != 0) { - synOut.update(index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize)) + synOut += ((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) } if (syn1Modify(index) != 0) { - synOut.update(index + vocabSize, - syn1Local.slice(index * vectorSize, (index + 1) * vectorSize)) + synOut += ((index + vocabSize, + syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) } index += 1 } - Iterator(synOut) + synOut.toIterator } - val synAgg = partial.flatMap(x => x).reduceByKey { case (v1, v2) => + val synAgg = partial.reduceByKey { case (v1, v2) => blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) v1 }.collect() From d75464de53b1550d99abf9b085181dc72ce837a7 Mon Sep 17 00:00:00 2001 From: freeman Date: Tue, 19 Aug 2014 18:07:42 -0700 Subject: [PATCH 050/192] [SPARK-3112][MLLIB] Add documentation and example for StreamingLR Added a documentation section on StreamingLR to the ``MLlib - Linear Methods``, including a worked example. mengxr tdas Author: freeman Closes #2047 from freeman-lab/streaming-lr-docs and squashes the following commits: 568d250 [freeman] Tweaks to wording / formatting 05a1139 [freeman] Added documentation and example for StreamingLR (cherry picked from commit c7252b0097cfacd36f17357d195b12a59e503b35) Signed-off-by: Xiangrui Meng --- docs/mllib-linear-methods.md | 75 ++++++++++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index e504cd7f0f578..9137f9dc1b692 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -518,6 +518,81 @@ print("Mean Squared Error = " + str(MSE)) +## Streaming linear regression + +When data arrive in a streaming fashion, it is useful to fit regression models online, +updating the parameters of the model as new data arrives. MLlib currently supports +streaming linear regression using ordinary least squares. The fitting is similar +to that performed offline, except fitting occurs on each batch of data, so that +the model continually updates to reflect the data from the stream. + +### Examples + +The following example demonstrates how to load training and testing data from two different +input streams of text files, parse the streams as labeled points, fit a linear regression model +online to the first stream, and make predictions on the second stream. + +
    + +
    + +First, we import the necessary classes for parsing our input data and creating the model. + +{% highlight scala %} + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD + +{% endhighlight %} + +Then we make input streams for training and testing data. We assume a StreamingContext `ssc` +has already been created, see [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) +for more info. For this example, we use labeled points in training and testing streams, +but in practice you will likely want to use unlabeled vectors for test data. + +{% highlight scala %} + +val trainingData = ssc.textFileStream('/training/data/dir').map(LabeledPoint.parse) +val testData = ssc.textFileStream('/testing/data/dir').map(LabeledPoint.parse) + +{% endhighlight %} + +We create our model by initializing the weights to 0 + +{% highlight scala %} + +val numFeatures = 3 +val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.zeros(numFeatures)) + +{% endhighlight %} + +Now we register the streams for training and testing and start the job. +Printing predictions alongside true labels lets us easily see the result. + +{% highlight scala %} + +model.trainOn(trainingData) +model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() + +ssc.start() +ssc.awaitTermination() + +{% endhighlight %} + +We can now save text files with data to the training or testing folders. +Each line should be a data point formatted as `(y,[x1,x2,x3])` where `y` is the label +and `x1,x2,x3` are the features. Anytime a text file is placed in `/training/data/dir` +the model will update. Anytime a text file is placed in `/testing/data/dir` you will see predictions. +As you feed more data to the training directory, the predictions +will get better! + +
    + +
    + + ## Implementation (developer) Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent From 607735c16b39ea89a11c2a0db38ae7d3422203d6 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 19 Aug 2014 19:37:02 -0700 Subject: [PATCH 051/192] [SQL] add note of use synchronizedMap in SQLConf Refer to: http://stackoverflow.com/questions/510632/whats-the-difference-between-concurrenthashmap-and-collections-synchronizedmap Collections.synchronizedMap(map) creates a blocking Map which will degrade performance, albeit ensure consistency. So use ConcurrentHashMap(a more effective thread-safe hashmap) instead. also update HiveQuerySuite to fix test error when changed to ConcurrentHashMap. Author: wangfei Author: scwf Closes #1996 from scwf/sqlconf and squashes the following commits: 93bc0c5 [wangfei] revert change of HiveQuerySuite 0cc05dd [wangfei] add note for use synchronizedMap 3c224d31 [scwf] fix formate a7bcb98 [scwf] use ConcurrentHashMap in sql conf, intead synchronizedMap (cherry picked from commit 0e3ab94d413fd70fff748fded42ab5e2ebd66fcc) Signed-off-by: Reynold Xin --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4f2adb006fbc7..5cc41a83cc792 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -54,6 +54,7 @@ private[spark] object SQLConf { trait SQLConf { import SQLConf._ + /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ @transient protected[spark] val settings = java.util.Collections.synchronizedMap( new java.util.HashMap[String, String]()) From 148e45b91aa4efcc0a7e5b28badff22887a92805 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 21:01:23 -0700 Subject: [PATCH 052/192] [SPARK-3130][MLLIB] detect negative values in naive Bayes because NB treats feature values as term frequencies. jkbradley Author: Xiangrui Meng Closes #2038 from mengxr/nb-neg and squashes the following commits: 52c37c3 [Xiangrui Meng] address comments 65f892d [Xiangrui Meng] detect negative values in nb (cherry picked from commit 068b6fe6a10eb1c6b2102d88832203267f030e85) Signed-off-by: Xiangrui Meng --- docs/mllib-naive-bayes.md | 3 +- .../mllib/classification/NaiveBayes.scala | 28 +++++++++++++++---- .../classification/NaiveBayesSuite.scala | 28 +++++++++++++++++++ 3 files changed, 53 insertions(+), 6 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 86d94aebd9442..7f9d4c6563944 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -17,7 +17,8 @@ Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bay which is typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each -feature represents a term whose value is the frequency of the term. +feature represents a term whose value is the frequency of the term. +Feature values must be nonnegative to represent term frequencies. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of 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 6c7be0a4f1dcb..8c8e4a161aa5b 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 @@ -19,9 +19,9 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -73,7 +73,7 @@ class NaiveBayesModel private[mllib] ( * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. */ class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { @@ -91,12 +91,30 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { + val requireNonnegativeValues: Vector => Unit = (v: Vector) => { + val values = v match { + case sv: SparseVector => + sv.values + case dv: DenseVector => + dv.values + } + if (!values.forall(_ >= 0.0)) { + throw new SparkException(s"Naive Bayes requires nonnegative feature values but found $v.") + } + } + // Aggregates term frequencies per label. // TODO: Calling combineByKey and collect creates two stages, we can implement something // TODO: similar to reduceByKeyLocally to save one stage. val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])]( - createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), - mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), + createCombiner = (v: Vector) => { + requireNonnegativeValues(v) + (1L, v.toBreeze.toDenseVector) + }, + mergeValue = (c: (Long, BDV[Double]), v: Vector) => { + requireNonnegativeValues(v) + (c._1 + 1L, c._2 += v.toBreeze) + }, mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => (c1._1 + c2._1, c1._2 += c2._2) ).collect() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 06cdd04f5fdae..80989bc074e84 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} @@ -95,6 +96,33 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("detect negative values") { + val dense = Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(-1.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(0.0))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(dense, 2)) + } + val sparse = Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(-1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(sparse, 2)) + } + val nan = Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(Double.NaN))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(nan, 2)) + } + } } class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { From d5db95baec62d911c7611f28535f0440440226cb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 22:05:29 -0700 Subject: [PATCH 053/192] [HOTFIX][Streaming][MLlib] use temp folder for checkpoint or Jenkins will complain about no Apache header in checkpoint files. tdas rxin Author: Xiangrui Meng Closes #2046 from mengxr/tmp-checkpoint and squashes the following commits: 0d3ec73 [Xiangrui Meng] remove ssc.stop 9797843 [Xiangrui Meng] change checkpointDir to lazy val 89964ab [Xiangrui Meng] use temp folder for checkpoint (cherry picked from commit fce5c0fb6384f3a142a4155525a5d62640725150) Signed-off-by: Xiangrui Meng --- .../StreamingLinearRegressionSuite.scala | 6 ------ .../apache/spark/streaming/TestSuiteBase.scala | 17 +++++++++++------ 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 28489410f8225..03b71301e9ab1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -49,7 +49,6 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data test("parameter accuracy") { - // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) @@ -82,7 +81,6 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // Test that parameter estimates improve when learning Y = 10*X1 on streaming data test("parameter convergence") { - // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) @@ -113,12 +111,10 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) // check that error shrunk on at least 2 batches assert(deltas.map(x => if ((x._1 - x._2) < 0) 1 else 0).sum > 1) - } // Test predictions on a stream test("predictions") { - // create model initialized with true weights val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(10.0, 10.0)) @@ -142,7 +138,5 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // compute the mean absolute error and check that it's always less than 0.1 val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) assert(errors.forall(x => x <= 0.1)) - } - } 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 f095da9cb55d3..759baacaa4308 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -17,18 +17,18 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} -import org.apache.spark.streaming.util.ManualClock +import java.io.{ObjectInputStream, IOException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.SynchronizedBuffer import scala.reflect.ClassTag -import java.io.{ObjectInputStream, IOException} - import org.scalatest.{BeforeAndAfter, FunSuite} +import com.google.common.io.Files -import org.apache.spark.{SparkContext, SparkConf, Logging} +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD /** @@ -119,7 +119,12 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def batchDuration = Seconds(1) // Directory where the checkpoint data will be saved - def checkpointDir = "checkpoint" + lazy val checkpointDir = { + val dir = Files.createTempDir() + logDebug(s"checkpointDir: $dir") + dir.deleteOnExit() + dir.toString + } // Number of partitions of the input parallel collections created for testing def numInputPartitions = 2 From 08c9973da01620c3592eac46d2437b18c4d5cba7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Aug 2014 22:11:13 -0700 Subject: [PATCH 054/192] [SPARK-3119] Re-implementation of TorrentBroadcast. This is a re-implementation of TorrentBroadcast, with the following changes: 1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched). 2. Removes TorrentInfo and TorrentBlock 3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block. 4. Removes the metadata block, resulting in one less block to fetch. 5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream). Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1). Old TorrentBroadcast for object of a single block: 1 RPC to ask for location of the broadcast variable 1 RPC to ask for location of the metadata block 1 RPC to fetch the metadata block 1 RPC to ask for location of the first data block 1 RPC to fetch the first data block 1 RPC to tell the driver we put the first data block in i.e. 5 + 1 New TorrentBroadcast for object of a single block: 1 RPC to ask for location of the first data block 1 RPC to get the first data block 1 RPC to tell the driver we put the first data block in i.e. 2 + 1 Author: Reynold Xin Closes #2030 from rxin/torrentBroadcast and squashes the following commits: 5bacb9d [Reynold Xin] Always add the object to driver's block manager. 0d8ed5b [Reynold Xin] Added getBytes to BlockManager and uses that in TorrentBroadcast. 2d6a5fb [Reynold Xin] Use putBytes/getRemoteBytes throughout. 3670f00 [Reynold Xin] Code review feedback. c1185cd [Reynold Xin] [SPARK-3119] Re-implementation of TorrentBroadcast. (cherry picked from commit 8adfbc2b6b5b647e450d30f89c141f935b6aa94b) Signed-off-by: Reynold Xin --- .../spark/broadcast/BroadcastFactory.scala | 11 + .../spark/broadcast/TorrentBroadcast.scala | 282 +++++++----------- .../spark/broadcast/BroadcastSuite.scala | 128 ++++---- 3 files changed, 181 insertions(+), 240 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index a8c827030a1ef..6a187b40628a2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -32,8 +32,19 @@ import org.apache.spark.annotation.DeveloperApi */ @DeveloperApi trait BroadcastFactory { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit + + /** + * Creates a new broadcast variable. + * + * @param value value to broadcast + * @param isLocal whether we are in local mode (single JVM process) + * @param id unique id representing this broadcast variable + */ def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit + def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index d8be649f96e5f..6173fd3a69fc7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -18,50 +18,116 @@ package org.apache.spark.broadcast import java.io._ +import java.nio.ByteBuffer +import scala.collection.JavaConversions.asJavaEnumeration import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} +import org.apache.spark.util.ByteBufferInputStream /** - * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like - * protocol to do a distributed transfer of the broadcasted data to the executors. - * The mechanism is as follows. The driver divides the serializes the broadcasted data, - * divides it into smaller chunks, and stores them in the BlockManager of the driver. - * These chunks are reported to the BlockManagerMaster so that all the executors can - * learn the location of those chunks. The first time the broadcast variable (sent as - * part of task) is deserialized at a executor, all the chunks are fetched using - * the BlockManager. When all the chunks are fetched (initially from the driver's - * BlockManager), they are combined and deserialized to recreate the broadcasted data. - * However, the chunks are also stored in the BlockManager and reported to the - * BlockManagerMaster. As more executors fetch the chunks, BlockManagerMaster learns - * multiple locations for each chunk. Hence, subsequent fetches of each chunk will be - * made to other executors who already have those chunks, resulting in a distributed - * fetching. This prevents the driver from being the bottleneck in sending out multiple - * copies of the broadcast data (one per executor) as done by the - * [[org.apache.spark.broadcast.HttpBroadcast]]. + * A BitTorrent-like implementation of [[org.apache.spark.broadcast.Broadcast]]. + * + * The mechanism is as follows: + * + * The driver divides the serialized object into small chunks and + * stores those chunks in the BlockManager of the driver. + * + * On each executor, the executor first attempts to fetch the object from its BlockManager. If + * it does not exist, it then uses remote fetches to fetch the small chunks from the driver and/or + * other executors if available. Once it gets the chunks, it puts the chunks in its own + * BlockManager, ready for other executors to fetch from. + * + * This prevents the driver from being the bottleneck in sending out multiple copies of the + * broadcast data (one per executor) as done by the [[org.apache.spark.broadcast.HttpBroadcast]]. + * + * @param obj object to broadcast + * @param isLocal whether Spark is running in local mode (single JVM process). + * @param id A unique identifier for the broadcast variable. */ private[spark] class TorrentBroadcast[T: ClassTag]( - @transient var value_ : T, isLocal: Boolean, id: Long) + obj : T, + @transient private val isLocal: Boolean, + id: Long) extends Broadcast[T](id) with Logging with Serializable { - override protected def getValue() = value_ + /** + * Value of the broadcast object. On driver, this is set directly by the constructor. + * On executors, this is reconstructed by [[readObject]], which builds this value by reading + * blocks from the driver and/or other executors. + */ + @transient private var _value: T = obj private val broadcastId = BroadcastBlockId(id) - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + /** Total number of blocks this broadcast variable contains. */ + private val numBlocks: Int = writeBlocks() + + override protected def getValue() = _value + + /** + * Divide the object into multiple blocks and put those blocks in the block manager. + * + * @return number of blocks this broadcast variable is divided into + */ + private def writeBlocks(): Int = { + // For local mode, just put the object in the BlockManager so we can find it later. + SparkEnv.get.blockManager.putSingle( + broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + + if (!isLocal) { + val blocks = TorrentBroadcast.blockifyObject(_value) + blocks.zipWithIndex.foreach { case (block, i) => + SparkEnv.get.blockManager.putBytes( + BroadcastBlockId(id, "piece" + i), + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) + } + blocks.length + } else { + 0 + } + } + + /** Fetch torrent blocks from the driver and/or other executors. */ + private def readBlocks(): Array[ByteBuffer] = { + // Fetch chunks of data. Note that all these chunks are stored in the BlockManager and reported + // to the driver, so other executors can pull these chunks from this executor as well. + val blocks = new Array[ByteBuffer](numBlocks) + val bm = SparkEnv.get.blockManager - @transient private var arrayOfBlocks: Array[TorrentBlock] = null - @transient private var totalBlocks = -1 - @transient private var totalBytes = -1 - @transient private var hasBlocks = 0 + for (pid <- Random.shuffle(Seq.range(0, numBlocks))) { + val pieceId = BroadcastBlockId(id, "piece" + pid) - if (!isLocal) { - sendBroadcast() + // First try getLocalBytes because there is a chance that previous attempts to fetch the + // broadcast blocks have already fetched some of the blocks. In that case, some blocks + // would be available locally (on this executor). + var blockOpt = bm.getLocalBytes(pieceId) + if (!blockOpt.isDefined) { + blockOpt = bm.getRemoteBytes(pieceId) + blockOpt match { + case Some(block) => + // If we found the block from remote executors/driver's BlockManager, put the block + // in this executor's BlockManager. + SparkEnv.get.blockManager.putBytes( + pieceId, + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) + + case None => + throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) + } + } + // If we get here, the option is defined. + blocks(pid) = blockOpt.get + } + blocks } /** @@ -79,26 +145,6 @@ private[spark] class TorrentBroadcast[T: ClassTag]( TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) } - private def sendBroadcast() { - val tInfo = TorrentBroadcast.blockifyObject(value_) - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - hasBlocks = tInfo.totalBlocks - - // Store meta-info - val metaId = BroadcastBlockId(id, "meta") - val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) - SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - - // Store individual pieces - for (i <- 0 until totalBlocks) { - val pieceId = BroadcastBlockId(id, "piece" + i) - SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } - } - /** Used by the JVM when serializing this object. */ private def writeObject(out: ObjectOutputStream) { assertValid() @@ -109,99 +155,30 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(broadcastId) match { + SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - value_ = x.asInstanceOf[T] + _value = x.asInstanceOf[T] case None => - val start = System.nanoTime logInfo("Started reading broadcast variable " + id) - - // Initialize @transient variables that will receive garbage values from the master. - resetWorkerVariables() - - if (receiveBroadcast()) { - value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - - /* Store the merged copy in cache so that the next worker doesn't need to rebuild it. - * This creates a trade-off between memory usage and latency. Storing copy doubles - * the memory footprint; not storing doubles deserialization cost. Also, - * this does not need to be reported to BlockManagerMaster since other executors - * does not need to access this block (they only need to fetch the chunks, - * which are reported). - */ - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - - // Remove arrayOfBlocks from memory once value_ is on local cache - resetWorkerVariables() - } else { - logError("Reading broadcast variable " + id + " failed") - } - - val time = (System.nanoTime - start) / 1e9 + val start = System.nanoTime() + val blocks = readBlocks() + val time = (System.nanoTime() - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") - } - } - } - - private def resetWorkerVariables() { - arrayOfBlocks = null - totalBytes = -1 - totalBlocks = -1 - hasBlocks = 0 - } - - private def receiveBroadcast(): Boolean = { - // Receive meta-info about the size of broadcast data, - // the number of chunks it is divided into, etc. - val metaId = BroadcastBlockId(id, "meta") - var attemptId = 10 - while (attemptId > 0 && totalBlocks == -1) { - SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => - val tInfo = x.asInstanceOf[TorrentInfo] - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - arrayOfBlocks = new Array[TorrentBlock](totalBlocks) - hasBlocks = 0 - case None => - Thread.sleep(500) - } - attemptId -= 1 - } - - if (totalBlocks == -1) { - return false - } - - /* - * Fetch actual chunks of data. Note that all these chunks are stored in - * the BlockManager and reported to the master, so that other executors - * can find out and pull the chunks from this executor. - */ - val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) - for (pid <- recvOrder) { - val pieceId = BroadcastBlockId(id, "piece" + pid) - SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => - arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] - hasBlocks += 1 + _value = TorrentBroadcast.unBlockifyObject[T](blocks) + // Store the merged copy in BlockManager so other tasks on this executor don't + // need to re-fetch it. SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - - case None => - throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) + broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } } - - hasBlocks == totalBlocks } - } -private[broadcast] object TorrentBroadcast extends Logging { + +private object TorrentBroadcast extends Logging { + /** Size of each block. Default value is 4MB. */ private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null @@ -223,7 +200,9 @@ private[broadcast] object TorrentBroadcast extends Logging { initialized = false } - def blockifyObject[T: ClassTag](obj: T): TorrentInfo = { + def blockifyObject[T: ClassTag](obj: T): Array[ByteBuffer] = { + // TODO: Create a special ByteArrayOutputStream that splits the output directly into chunks + // so we don't need to do the extra memory copy. val bos = new ByteArrayOutputStream() val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos val ser = SparkEnv.get.serializer.newInstance() @@ -231,44 +210,27 @@ private[broadcast] object TorrentBroadcast extends Logging { serOut.writeObject[T](obj).close() val byteArray = bos.toByteArray val bais = new ByteArrayInputStream(byteArray) + val numBlocks = math.ceil(byteArray.length.toDouble / BLOCK_SIZE).toInt + val blocks = new Array[ByteBuffer](numBlocks) - var blockNum = byteArray.length / BLOCK_SIZE - if (byteArray.length % BLOCK_SIZE != 0) { - blockNum += 1 - } - - val blocks = new Array[TorrentBlock](blockNum) var blockId = 0 - for (i <- 0 until (byteArray.length, BLOCK_SIZE)) { val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i) val tempByteArray = new Array[Byte](thisBlockSize) bais.read(tempByteArray, 0, thisBlockSize) - blocks(blockId) = new TorrentBlock(blockId, tempByteArray) + blocks(blockId) = ByteBuffer.wrap(tempByteArray) blockId += 1 } bais.close() - - val info = TorrentInfo(blocks, blockNum, byteArray.length) - info.hasBlocks = blockNum - info + blocks } - def unBlockifyObject[T: ClassTag]( - arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, - totalBlocks: Int): T = { - val retByteArray = new Array[Byte](totalBytes) - for (i <- 0 until totalBlocks) { - System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, - i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length) - } + def unBlockifyObject[T: ClassTag](blocks: Array[ByteBuffer]): T = { + val is = new SequenceInputStream( + asJavaEnumeration(blocks.iterator.map(block => new ByteBufferInputStream(block)))) + val in: InputStream = if (compress) compressionCodec.compressedInputStream(is) else is - val in: InputStream = { - val arrIn = new ByteArrayInputStream(retByteArray) - if (compress) compressionCodec.compressedInputStream(arrIn) else arrIn - } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) val obj = serIn.readObject[T]() @@ -284,17 +246,3 @@ private[broadcast] object TorrentBroadcast extends Logging { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } } - -private[broadcast] case class TorrentBlock( - blockID: Int, - byteArray: Array[Byte]) - extends Serializable - -private[broadcast] case class TorrentInfo( - @transient arrayOfBlocks: Array[TorrentBlock], - totalBlocks: Int, - totalBytes: Int) - extends Serializable { - - @transient var hasBlocks = 0 -} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 17c64455b2429..978a6ded80829 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.broadcast -import org.apache.spark.storage.{BroadcastBlockId, _} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.scalatest.FunSuite +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} +import org.apache.spark.storage._ + + class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -124,12 +126,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { val numSlaves = if (distributed) 2 else 0 - def getBlockIds(id: Long) = Seq[BroadcastBlockId](BroadcastBlockId(id)) - // Verify that the broadcast file is created, and blocks are persisted only on the driver - def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterCreation(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === 1) statuses.head match { case (bm, status) => assert(bm.executorId === "", "Block should only be on the driver") @@ -139,14 +139,14 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } if (distributed) { // this file is only generated in distributed mode - assert(HttpBroadcast.getFile(blockIds.head.broadcastId).exists, "Broadcast file not found!") + assert(HttpBroadcast.getFile(blockId.broadcastId).exists, "Broadcast file not found!") } } // Verify that blocks are persisted in both the executors and the driver - def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === numSlaves + 1) statuses.foreach { case (_, status) => assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) @@ -157,21 +157,21 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver // is true. In the latter case, also verify that the broadcast file is deleted on the driver. - def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) val expectedNumBlocks = if (removeFromDriver) 0 else 1 val possiblyNot = if (removeFromDriver) "" else " not" assert(statuses.size === expectedNumBlocks, "Block should%s be unpersisted on the driver".format(possiblyNot)) if (distributed && removeFromDriver) { // this file is only generated in distributed mode - assert(!HttpBroadcast.getFile(blockIds.head.broadcastId).exists, + assert(!HttpBroadcast.getFile(blockId.broadcastId).exists, "Broadcast file should%s be deleted".format(possiblyNot)) } } - testUnpersistBroadcast(distributed, numSlaves, httpConf, getBlockIds, afterCreation, + testUnpersistBroadcast(distributed, numSlaves, httpConf, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -185,67 +185,51 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean) { val numSlaves = if (distributed) 2 else 0 - def getBlockIds(id: Long) = { - val broadcastBlockId = BroadcastBlockId(id) - val metaBlockId = BroadcastBlockId(id, "meta") - // Assume broadcast value is small enough to fit into 1 piece - val pieceBlockId = BroadcastBlockId(id, "piece0") - if (distributed) { - // the metadata and piece blocks are generated only in distributed mode - Seq[BroadcastBlockId](broadcastBlockId, metaBlockId, pieceBlockId) - } else { - Seq[BroadcastBlockId](broadcastBlockId) - } + // Verify that blocks are persisted only on the driver + def afterCreation(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === 1) + + blockId = BroadcastBlockId(broadcastId, "piece0") + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === (if (distributed) 1 else 0)) } - // Verify that blocks are persisted only on the driver - def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (distributed) { + assert(statuses.size === numSlaves + 1) + } else { assert(statuses.size === 1) - statuses.head match { case (bm, status) => - assert(bm.executorId === "", "Block should only be on the driver") - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store on the driver") - assert(status.diskSize === 0, "Block should not be in disk store on the driver") - } } - } - // Verify that blocks are persisted in both the executors and the driver - def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - if (blockId.field == "meta") { - // Meta data is only on the driver - assert(statuses.size === 1) - statuses.head match { case (bm, _) => assert(bm.executorId === "") } - } else { - // Other blocks are on both the executors and the driver - assert(statuses.size === numSlaves + 1, - blockId + " has " + statuses.size + " statuses: " + statuses.mkString(",")) - statuses.foreach { case (_, status) => - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store") - assert(status.diskSize === 0, "Block should not be in disk store") - } - } + blockId = BroadcastBlockId(broadcastId, "piece0") + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (distributed) { + assert(statuses.size === numSlaves + 1) + } else { + assert(statuses.size === 0) } } // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver // is true. - def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - val expectedNumBlocks = if (removeFromDriver) 0 else 1 - val possiblyNot = if (removeFromDriver) "" else " not" - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === expectedNumBlocks, - "Block should%s be unpersisted on the driver".format(possiblyNot)) - } + def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var expectedNumBlocks = if (removeFromDriver) 0 else 1 + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks) + + blockId = BroadcastBlockId(broadcastId, "piece0") + expectedNumBlocks = if (removeFromDriver || !distributed) 0 else 1 + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks) } - testUnpersistBroadcast(distributed, numSlaves, torrentConf, getBlockIds, afterCreation, + testUnpersistBroadcast(distributed, numSlaves, torrentConf, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -262,10 +246,9 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { distributed: Boolean, numSlaves: Int, // used only when distributed = true broadcastConf: SparkConf, - getBlockIds: Long => Seq[BroadcastBlockId], - afterCreation: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, - afterUsingBroadcast: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, - afterUnpersist: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterCreation: (Long, BlockManagerMaster) => Unit, + afterUsingBroadcast: (Long, BlockManagerMaster) => Unit, + afterUnpersist: (Long, BlockManagerMaster) => Unit, removeFromDriver: Boolean) { sc = if (distributed) { @@ -278,15 +261,14 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { // Create broadcast variable val broadcast = sc.broadcast(list) - val blocks = getBlockIds(broadcast.id) - afterCreation(blocks, blockManagerMaster) + afterCreation(broadcast.id, blockManagerMaster) // Use broadcast variable on all executors val partitions = 10 assert(partitions > numSlaves) val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) - afterUsingBroadcast(blocks, blockManagerMaster) + afterUsingBroadcast(broadcast.id, blockManagerMaster) // Unpersist broadcast if (removeFromDriver) { @@ -294,7 +276,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } else { broadcast.unpersist(blocking = true) } - afterUnpersist(blocks, blockManagerMaster) + afterUnpersist(broadcast.id, blockManagerMaster) // If the broadcast is removed from driver, all subsequent uses of the broadcast variable // should throw SparkExceptions. Otherwise, the result should be the same as before. From a5bc9c601e9093b3b896563d23bb2e4add1f0676 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 22:16:22 -0700 Subject: [PATCH 055/192] [SPARK-3142][MLLIB] output shuffle data directly in Word2Vec Sorry I didn't realize this in #2043. Ishiihara Author: Xiangrui Meng Closes #2049 from mengxr/more-w2v and squashes the following commits: 050b1c5 [Xiangrui Meng] output shuffle data directly (cherry picked from commit 0a984aa155fb7f532fe87620dcf1a2814c5b8b49) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/feature/Word2Vec.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index c3375ed44fd99..fc1444705364a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -347,19 +347,20 @@ class Word2Vec extends Serializable with Logging { } val syn0Local = model._1 val syn1Local = model._2 - val synOut = mutable.ListBuffer.empty[(Int, Array[Float])] - var index = 0 - while(index < vocabSize) { - if (syn0Modify(index) != 0) { - synOut += ((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) + // Only output modified vectors. + Iterator.tabulate(vocabSize) { index => + if (syn0Modify(index) > 0) { + Some((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) + } else { + None } - if (syn1Modify(index) != 0) { - synOut += ((index + vocabSize, - syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) + }.flatten ++ Iterator.tabulate(vocabSize) { index => + if (syn1Modify(index) > 0) { + Some((index + vocabSize, syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) + } else { + None } - index += 1 - } - synOut.toIterator + }.flatten } val synAgg = partial.reduceByKey { case (v1, v2) => blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) From 5d1a8786686705ae494f60a47c3a9c2e0ce8ff14 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 19 Aug 2014 22:42:50 -0700 Subject: [PATCH 056/192] [SPARK-2974] [SPARK-2975] Fix two bugs related to spark.local.dirs This PR fixes two bugs related to `spark.local.dirs` and `SPARK_LOCAL_DIRS`, one where `Utils.getLocalDir()` might return an invalid directory (SPARK-2974) and another where the `SPARK_LOCAL_DIRS` override didn't affect the driver, which could cause problems when running tasks in local mode (SPARK-2975). This patch fixes both issues: the new `Utils.getOrCreateLocalRootDirs(conf: SparkConf)` utility method manages the creation of local directories and handles the precedence among the different configuration options, so we should see the same behavior whether we're running in local mode or on a worker. It's kind of a pain to mock out environment variables in tests (no easy way to mock System.getenv), so I added a `private[spark]` method to SparkConf for accessing environment variables (by default, it just delegates to System.getenv). By subclassing SparkConf and overriding this method, we can mock out SPARK_LOCAL_DIRS in tests. I also fixed a typo in PySpark where we used `SPARK_LOCAL_DIR` instead of `SPARK_LOCAL_DIRS` (I think this was technically innocuous, but it seemed worth fixing). Author: Josh Rosen Closes #2002 from JoshRosen/local-dirs and squashes the following commits: efad8c6 [Josh Rosen] Address review comments: 1dec709 [Josh Rosen] Minor updates to Javadocs. 7f36999 [Josh Rosen] Use env vars to detect if running in YARN container. 399ac25 [Josh Rosen] Update getLocalDir() documentation. bb3ad89 [Josh Rosen] Remove duplicated YARN getLocalDirs() code. 3e92d44 [Josh Rosen] Move local dirs override logic into Utils; fix bugs: b2c4736 [Josh Rosen] Add failing tests for SPARK-2974 and SPARK-2975. 007298b [Josh Rosen] Allow environment variables to be mocked in tests. 6d9259b [Josh Rosen] Fix typo in PySpark: SPARK_LOCAL_DIR should be SPARK_LOCAL_DIRS (cherry picked from commit ebcb94f701273b56851dade677e047388a8bca09) Signed-off-by: Patrick Wendell --- .../scala/org/apache/spark/SparkConf.scala | 8 ++- .../apache/spark/api/python/PythonRDD.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 25 ------- .../apache/spark/storage/BlockManager.scala | 3 +- .../spark/storage/DiskBlockManager.scala | 14 ++-- .../scala/org/apache/spark/util/Utils.scala | 67 +++++++++++++++++-- .../spark/storage/BlockManagerSuite.scala | 3 +- .../spark/storage/DiskBlockManagerSuite.scala | 4 +- .../apache/spark/storage/LocalDirsSuite.scala | 61 +++++++++++++++++ python/pyspark/shuffle.py | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 18 ----- .../spark/deploy/yarn/ExecutorLauncher.scala | 19 ------ .../spark/deploy/yarn/ApplicationMaster.scala | 18 ----- .../spark/deploy/yarn/ExecutorLauncher.scala | 19 ------ 14 files changed, 145 insertions(+), 118 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b4f321ec99e78..605df0e929faa 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -45,7 +45,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) - private val settings = new HashMap[String, String]() + private[spark] val settings = new HashMap[String, String]() if (loadDefaults) { // Load any spark.* system properties @@ -210,6 +210,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** + * By using this instead of System.getenv(), environment variables can be mocked + * in unit tests. + */ + private[spark] def getenv(name: String): String = System.getenv(name) + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ private[spark] def validateSettings() { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 10210a2927dcc..747023812f754 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -62,7 +62,7 @@ private[spark] class PythonRDD( val env = SparkEnv.get val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") - envVars += ("SPARK_LOCAL_DIR" -> localdir) // it's also used in monitor thread + envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index fb3f7bd54bbfa..2f76e532aeb76 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -62,16 +62,6 @@ private[spark] class Executor( val conf = new SparkConf(true) conf.setAll(properties) - // If we are in yarn mode, systems can have different disk layouts so we must set it - // to what Yarn on this system said was available. This will be used later when SparkEnv - // created. - if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { - conf.set("spark.local.dir", getYarnLocalDirs()) - } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { - conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) - } - if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire @@ -134,21 +124,6 @@ private[spark] class Executor( threadPool.shutdown() } - /** Get the Yarn approved local directories. */ - private def getYarnLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) - - if (localDirs.isEmpty) { - throw new Exception("Yarn Local dirs can't be empty") - } - localDirs - } - class TaskRunner( execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { 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 c0491fb55e3a4..12a92d44f4c36 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -67,8 +67,7 @@ private[spark] class BlockManager( private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) - val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) val connectionManager = new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f3da816389581..ec022ce9c048a 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -21,7 +21,7 @@ import java.io.File import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} -import org.apache.spark.{SparkEnv, Logging} +import org.apache.spark.{SparkConf, SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.network.netty.PathResolver import org.apache.spark.util.Utils @@ -33,9 +33,10 @@ import org.apache.spark.shuffle.sort.SortShuffleManager * However, it is also possible to have a block map to only a segment of a file, by calling * mapBlockToFileSegment(). * - * @param rootDirs The directories to use for storing block files. Data will be hashed among these. + * Block files are hashed among the directories listed in spark.local.dir (or in + * SPARK_LOCAL_DIRS, if it's set). */ -private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, rootDirs: String) +private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, conf: SparkConf) extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -46,7 +47,7 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid * having really large inodes at the top level. */ - val localDirs: Array[File] = createLocalDirs() + val localDirs: Array[File] = createLocalDirs(conf) if (localDirs.isEmpty) { logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) @@ -130,10 +131,9 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, (blockId, getFile(blockId)) } - private def createLocalDirs(): Array[File] = { - logDebug(s"Creating local directories at root dirs '$rootDirs'") + private def createLocalDirs(conf: SparkConf): Array[File] = { val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").flatMap { rootDir => + Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir => var foundLocalDir = false var localDir: File = null var localDirId: String = null 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 019f68b160894..d6d74ce269219 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -449,12 +449,71 @@ private[spark] object Utils extends Logging { } /** - * Get a temporary directory using Spark's spark.local.dir property, if set. This will always - * return a single directory, even though the spark.local.dir property might be a list of - * multiple paths. + * Get the path of a temporary directory. Spark's local directories can be configured through + * multiple settings, which are used with the following precedence: + * + * - If called from inside of a YARN container, this will return a directory chosen by YARN. + * - If the SPARK_LOCAL_DIRS environment variable is set, this will return a directory from it. + * - Otherwise, if the spark.local.dir is set, this will return a directory from it. + * - Otherwise, this will return java.io.tmpdir. + * + * Some of these configuration options might be lists of multiple paths, but this method will + * always return a single directory. */ def getLocalDir(conf: SparkConf): String = { - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + getOrCreateLocalRootDirs(conf)(0) + } + + private[spark] def isRunningInYarnContainer(conf: SparkConf): Boolean = { + // These environment variables are set by YARN. + // For Hadoop 0.23.X, we check for YARN_LOCAL_DIRS (we use this below in getYarnLocalDirs()) + // For Hadoop 2.X, we check for CONTAINER_ID. + conf.getenv("CONTAINER_ID") != null || conf.getenv("YARN_LOCAL_DIRS") != null + } + + /** + * Gets or creates the directories listed in spark.local.dir or SPARK_LOCAL_DIRS, + * and returns only the directories that exist / could be created. + * + * If no directories could be created, this will return an empty list. + */ + private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { + val confValue = if (isRunningInYarnContainer(conf)) { + // If we are in yarn mode, systems can have different disk layouts so we must set it + // to what Yarn on this system said was available. + getYarnLocalDirs(conf) + } else { + Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse( + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + } + val rootDirs = confValue.split(',') + logDebug(s"Getting/creating local root dirs at '$confValue'") + + rootDirs.flatMap { rootDir => + val localDir: File = new File(rootDir) + val foundLocalDir = localDir.exists || localDir.mkdirs() + if (!foundLocalDir) { + logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.") + None + } else { + Some(rootDir) + } + } + } + + /** Get the Yarn approved local directories. */ + private def getYarnLocalDirs(conf: SparkConf): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(conf.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(conf.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs } /** 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 20bac66105a69..f32ce6f9fcc7f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -825,8 +825,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter 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")) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 777579bc570db..aabaeadd7a071 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -71,7 +71,9 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before } override def beforeEach() { - diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) + val conf = testConf.clone + conf.set("spark.local.dir", rootDirs) + diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) shuffleBlockManager.idToSegmentMap.clear() } diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala new file mode 100644 index 0000000000000..dae7bf0e336de --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.File + +import org.apache.spark.util.Utils +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf + + +/** + * Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options. + */ +class LocalDirsSuite extends FunSuite { + + test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") { + // Regression test for SPARK-2974 + assert(!new File("/NONEXISTENT_DIR").exists()) + val conf = new SparkConf(false) + .set("spark.local.dir", s"/NONEXISTENT_PATH,${System.getProperty("java.io.tmpdir")}") + assert(new File(Utils.getLocalDir(conf)).exists()) + } + + test("SPARK_LOCAL_DIRS override also affects driver") { + // Regression test for SPARK-2975 + assert(!new File("/NONEXISTENT_DIR").exists()) + // SPARK_LOCAL_DIRS is a valid directory: + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir") + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(settings) + } + } + // spark.local.dir only contains invalid directories, but that's not a problem since + // SPARK_LOCAL_DIRS will override it on both the driver and workers: + val conf = new MySparkConf().set("spark.local.dir", "/NONEXISTENT_PATH") + assert(new File(Utils.getLocalDir(conf)).exists()) + } + +} diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 2c68cd4921deb..1ebe7df418327 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -214,7 +214,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, def _get_dirs(self): """ Get all the directories """ - path = os.environ.get("SPARK_LOCAL_DIR", "/tmp") + path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") dirs = path.split(",") return [os.path.join(d, "python", str(os.getpid()), str(id(self))) for d in dirs] diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 46a01f5a9a2cc..4d4848b1bd8f8 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -72,10 +72,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false def run() { - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - // set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box System.setProperty("spark.ui.port", "0") @@ -138,20 +134,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, params) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 72c7143edcd71..c3310fbc24a98 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -95,11 +95,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() @@ -152,20 +147,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp System.exit(0) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9c2bcf17a8508..1c4005fd8e78e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -72,10 +72,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false def run() { - // Setup the directories so things go to YARN approved directories rather - // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box System.setProperty("spark.ui.port", "0") @@ -144,20 +140,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) } - // Get the Yarn approved local directories. - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn local dirs can't be empty") - case Some(l) => l - } - } - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index a7585748b7f88..45925f1fea005 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -94,11 +94,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - amClient = AMRMClient.createAMRMClient() amClient.init(yarnConf) amClient.start() @@ -141,20 +136,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp System.exit(0) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") From f8c908ebfebb4b7a09dec6c806732997a73c1b84 Mon Sep 17 00:00:00 2001 From: Ken Takagiwa Date: Tue, 19 Aug 2014 22:43:22 -0700 Subject: [PATCH 057/192] [DOCS] Fixed wrong links Author: Ken Takagiwa Closes #2042 from giwa/patch-1 and squashes the following commits: 216fe0e [Ken Takagiwa] Fixed wrong links (cherry picked from commit 8a74e4b2a8c7dab154b406539487cf29d578d208) Signed-off-by: Reynold Xin --- docs/streaming-custom-receivers.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 1e045a3dd0ca9..27cd085782f66 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -186,7 +186,7 @@ JavaDStream words = lines.flatMap(new FlatMapFunction() ... {% endhighlight %} -The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java). +The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java). @@ -215,7 +215,7 @@ And a new input stream can be created with this custom actor as val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") {% endhighlight %} -See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala) +See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala) for an end-to-end example. From 5b22ebf68bdf7ac537999abb0e7d18c18ad8d0b0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 19 Aug 2014 22:43:49 -0700 Subject: [PATCH 058/192] [SPARK-3141] [PySpark] fix sortByKey() with take() Fix sortByKey() with take() The function `f` used in mapPartitions should always return an iterator. Author: Davies Liu Closes #2045 from davies/fix_sortbykey and squashes the following commits: 1160f59 [Davies Liu] fix sortByKey() with take() (cherry picked from commit 0a7ef6339f18e68d703599aff7db2dd9c2003866) Signed-off-by: Patrick Wendell --- python/pyspark/rdd.py | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 140cbe05a43b0..3eefc878d274e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -575,6 +575,8 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] + >>> sc.parallelize(tmp).sortByKey().first() + ('1', 3) >>> sc.parallelize(tmp).sortByKey(True, 1).collect() [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> sc.parallelize(tmp).sortByKey(True, 2).collect() @@ -587,14 +589,13 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() + def sortPartition(iterator): + return iter(sorted(iterator, key=lambda (k, v): keyfunc(k), reverse=not ascending)) + if numPartitions == 1: if self.getNumPartitions() > 1: self = self.coalesce(1) - - def sort(iterator): - return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - - return self.mapPartitions(sort) + return self.mapPartitions(sortPartition) # first compute the boundary of each part via sampling: we want to partition # the key-space into bins such that the bins have roughly the same @@ -610,17 +611,14 @@ def sort(iterator): bounds = [samples[len(samples) * (i + 1) / numPartitions] for i in range(0, numPartitions - 1)] - def rangePartitionFunc(k): + def rangePartitioner(k): p = bisect.bisect_left(bounds, keyfunc(k)) if ascending: return p else: return numPartitions - 1 - p - def mapFunc(iterator): - return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - - return self.partitionBy(numPartitions, rangePartitionFunc).mapPartitions(mapFunc, True) + return self.partitionBy(numPartitions, rangePartitioner).mapPartitions(sortPartition, True) def sortBy(self, keyfunc, ascending=True, numPartitions=None): """ From 9b29099557596356c2ae6baa82afc899c8a557f2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 20 Aug 2014 04:09:54 -0700 Subject: [PATCH 059/192] [SPARK-3054][STREAMING] Add unit tests for Spark Sink. This patch adds unit tests for Spark Sink. It also removes the private[flume] for Spark Sink, since the sink is instantiated from Flume configuration (looks like this is ignored by reflection which is used by Flume, but we should still remove it anyway). Author: Hari Shreedharan Author: Hari Shreedharan Closes #1958 from harishreedharan/spark-sink-test and squashes the following commits: e3110b9 [Hari Shreedharan] Add a sleep to allow sink to commit the transactions 120b81e [Hari Shreedharan] Fix complexity in threading model in test 4df5be6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test c9190d1 [Hari Shreedharan] Indentation and spaces changes 7fedc5a [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test abc20cb [Hari Shreedharan] Minor test changes 7b9b649 [Hari Shreedharan] Merge branch 'master' into spark-sink-test f2c56c9 [Hari Shreedharan] Update SparkSinkSuite.scala a24aac8 [Hari Shreedharan] Remove unused var c86d615 [Hari Shreedharan] [SPARK-3054][STREAMING] Add unit tests for Spark Sink. (cherry picked from commit 8c5a2226932c572898c76eb6fab9283f02ad4103) Signed-off-by: Tathagata Das --- external/flume-sink/pom.xml | 7 + .../streaming/flume/sink/SparkSink.scala | 1 - .../streaming/flume/sink/SparkSinkSuite.scala | 204 ++++++++++++++++++ .../flume/FlumePollingStreamSuite.scala | 2 +- 4 files changed, 212 insertions(+), 2 deletions(-) create mode 100644 external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index cfbf943bdafe0..7f1172ec2092d 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -72,6 +72,13 @@ org.scalatest scalatest_${scala.binary.version}
    + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + target/scala-${scala.binary.version}/classes diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 7b735133e3d14..1a61b36910a95 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -53,7 +53,6 @@ import org.apache.flume.sink.AbstractSink * */ -private[flume] class SparkSink extends AbstractSink with Logging with Configurable { // Size of the pool to use for holding transaction processors. diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala new file mode 100644 index 0000000000000..44b27edf85ce8 --- /dev/null +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -0,0 +1,204 @@ +/* + * 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.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} + +import scala.collection.JavaConversions._ +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.event.EventBuilder +import org.apache.spark.streaming.TestSuiteBase +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +class SparkSinkSuite extends TestSuiteBase { + val eventsPerBatch = 1000 + val channelCapacity = 5000 + + test("Success") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + client.ack(events.getSequenceNumber) + assert(events.getEvents.size() === 1000) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Nack") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + client.nack(events.getSequenceNumber) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Timeout") { + val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig + .CONF_TRANSACTION_TIMEOUT -> 1.toString)) + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + Thread.sleep(1000) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Multiple consumers") { + testMultipleConsumers(failSome = false) + } + + test("Multiple consumers with some failures") { + testMultipleConsumers(failSome = true) + } + + def testMultipleConsumers(failSome: Boolean): Unit = { + implicit val executorContext = ExecutionContext + .fromExecutorService(Executors.newFixedThreadPool(5)) + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + val transceiversAndClients = getTransceiverAndClient(address, 5) + val batchCounter = new CountDownLatch(5) + val counter = new AtomicInteger(0) + transceiversAndClients.foreach(x => { + Future { + val client = x._2 + val events = client.getEventBatch(1000) + if (!failSome || counter.getAndIncrement() % 2 == 0) { + client.ack(events.getSequenceNumber) + } else { + client.nack(events.getSequenceNumber) + throw new RuntimeException("Sending NACK for failure!") + } + events + }.onComplete { + case Success(events) => + assert(events.getEvents.size() === 1000) + batchCounter.countDown() + case Failure(t) => + // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout + batchCounter.countDown() + } + }) + batchCounter.await() + TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. + executorContext.shutdown() + if(failSome) { + assert(availableChannelSlots(channel) === 3000) + } else { + assertChannelIsEmpty(channel) + } + sink.stop() + channel.stop() + transceiversAndClients.foreach(x => x._1.close()) + } + + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, + SparkSink) = { + val channel = new MemoryChannel() + val channelContext = new Context() + + channelContext.put("capacity", channelCapacity.toString) + channelContext.put("transactionCapacity", 1000.toString) + channelContext.put("keep-alive", 0.toString) + channelContext.putAll(overrides) + channel.configure(channelContext) + + val sink = new SparkSink() + val sinkContext = new Context() + sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") + sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) + sink.configure(sinkContext) + sink.setChannel(channel) + (channel, sink) + } + + private def putEvents(ch: MemoryChannel, count: Int): Unit = { + val tx = ch.getTransaction + tx.begin() + (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) + tx.commit() + tx.close() + } + + private def getTransceiverAndClient(address: InetSocketAddress, + count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { + + (1 to count).map(_ => { + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + val transceiver = new NettyTransceiver(address, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + (transceiver, client) + }) + } + + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { + assert(availableChannelSlots(channel) === channelCapacity) + } + + private def availableChannelSlots(channel: MemoryChannel): Int = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 27bf2ac962721..2e4ac7cfbf263 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -171,7 +171,7 @@ class FlumePollingStreamSuite extends TestSuiteBase { } def assertChannelIsEmpty(channel: MemoryChannel) = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") assert(m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] === 5000) From ca7322dda10def28b1133876aa9196f555c5025e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 12:13:31 -0700 Subject: [PATCH 060/192] SPARK-3092 [SQL]: Always include the thriftserver when -Phive is enabled. Currently we have a separate profile called hive-thriftserver. I originally suggested this in case users did not want to bundle the thriftserver, but it's ultimately lead to a lot of confusion. Since the thriftserver is only a few classes, I don't see a really good reason to isolate it from the rest of Hive. So let's go ahead and just include it in the same profile to simplify things. This has been suggested in the past by liancheng. Author: Patrick Wendell Closes #2006 from pwendell/hiveserver and squashes the following commits: 742ea40 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into hiveserver 034ad47 [Patrick Wendell] SPARK-3092: Always include the thriftserver when -Phive is enabled. (cherry picked from commit f2f26c2a1dc6d60078c3be9c3d11a21866d9a24f) Signed-off-by: Patrick Wendell --- README.md | 6 +----- assembly/pom.xml | 5 ----- dev/create-release/create-release.sh | 10 +++++----- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/building-with-maven.md | 8 ++------ docs/sql-programming-guide.md | 4 +--- pom.xml | 2 +- 8 files changed, 12 insertions(+), 27 deletions(-) diff --git a/README.md b/README.md index a1a48f5bd0819..8906e4c1416b1 100644 --- a/README.md +++ b/README.md @@ -118,11 +118,7 @@ If your project is built with Maven, add this to your POM file's ` ## A Note About Thrift JDBC server and CLI for Spark SQL Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about those features. -You can use those features by setting `-Phive-thriftserver` when building Spark as follows. - - $ sbt/sbt -Phive-thriftserver assembly - +See sql-programming-guide.md for more information about using the JDBC server. ## Configuration diff --git a/assembly/pom.xml b/assembly/pom.xml index 16e5271b35050..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -163,11 +163,6 @@ spark-hive_${scala.binary.version} ${project.version}
    - - - - hive-thriftserver - org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 28f26d2368254..905dec0ced383 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -60,14 +60,14 @@ if [[ ! "$@" =~ --package-only ]]; then -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dmaven.javadoc.skip=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 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ release:perform cd .. @@ -117,10 +117,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & +make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & + "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & make_binary_release "hadoop2-without-hive" \ "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & wait diff --git a/dev/run-tests b/dev/run-tests index 132f696d6447a..20a67cfb361b9 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -99,7 +99,7 @@ echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, diff --git a/dev/scalastyle b/dev/scalastyle index b53053a04ff42..eb9b467965636 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 4d87ab92cec5b..a7d7bd3ccb1f2 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -98,12 +98,8 @@ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -Dski # Building Thrift JDBC server and CLI for Spark SQL -Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about those features. -You can use those features by setting `-Phive-thriftserver` when building Spark as follows. -{% highlight bash %} -mvn -Phive-thriftserver assembly -{% endhighlight %} +Spark SQL supports Thrift JDBC server and CLI. See sql-programming-guide.md for +more information about the JDBC server. # Spark Tests in Maven diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 34accade36ea9..c41f2804a6021 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -578,9 +578,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] (https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive -you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` -for maven). +the JDBC server with the beeline script comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: diff --git a/pom.xml b/pom.xml index 8c4c4af0eda8e..1479326af0ed9 100644 --- a/pom.xml +++ b/pom.xml @@ -1178,7 +1178,7 @@ - hive-thriftserver + hive false From 99ca704aba34282d97a8d05bc2b283a4b344bff2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 20 Aug 2014 12:57:39 -0700 Subject: [PATCH 061/192] [SPARK-3126][SPARK-3127][SQL] Fixed HiveThriftServer2Suite This PR fixes two issues: 1. Fixes wrongly quoted command line option in `HiveThriftServer2Suite` that makes test cases hang until timeout. 1. Asks `dev/run-test` to run Spark SQL tests when `bin/spark-sql` and/or `sbin/start-thriftserver.sh` are modified. Author: Cheng Lian Closes #2036 from liancheng/fix-thriftserver-test and squashes the following commits: f38c4eb [Cheng Lian] Fixed the same quotation issue in CliSuite 26b82a0 [Cheng Lian] Run SQL tests when dff contains bin/spark-sql and/or sbin/start-thriftserver.sh a87f83d [Cheng Lian] Extended timeout e5aa31a [Cheng Lian] Fixed metastore JDBC URI quotation (cherry picked from commit cf46e725814f575ebb417e80d2571bccc6dac4a7) Signed-off-by: Michael Armbrust --- dev/run-tests | 2 +- .../spark/sql/hive/thriftserver/CliSuite.scala | 2 +- .../thriftserver/HiveThriftServer2Suite.scala | 18 ++++-------------- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 20a67cfb361b9..d751961605dfd 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -55,7 +55,7 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* # Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --name-only master | grep "^sql/"` + diffs=`git diff --name-only master | grep "^\(sql/\)\|\(bin/spark-sql\)\|\(sbin/start-thriftserver.sh\)"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 2bf8cfdcacd22..70bea1ed80fda 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -32,7 +32,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { val commands = s"""../../bin/spark-sql | --master local - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH """.stripMargin.split("\\s+") diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index aedef6ce1f5f2..326b0a7275b34 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -51,9 +51,6 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt port } - // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. - val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean - Class.forName(DRIVER_NAME) override def beforeAll() { launchServer() } @@ -68,8 +65,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt val command = s"""../../sbin/start-thriftserver.sh | --master local - | --hiveconf hive.root.logger=INFO,console - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT @@ -77,12 +73,10 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt val pb = new ProcessBuilder(command ++ args: _*) val environment = pb.environment() - environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) - environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) process = pb.start() inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + waitForOutput(inputReader, "ThriftBinaryCLIService listening on", 300000) // Spawn a thread to read the output from the forked process. // Note that this is necessary since in some configurations, log4j could be blocked @@ -91,12 +85,8 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt while (true) { val stdout = readFrom(inputReader) val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } + print(stdout) + print(stderr) Thread.sleep(50) } } From 5095851fc284f31e7d91d192c88d1bbcf02e1d0e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 20 Aug 2014 13:26:11 -0700 Subject: [PATCH 062/192] [SPARK-3062] [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled #1891 was to avoid IOException when EventLogging is enabled. The solution used ShutdownHookManager but it was defined only Hadoop 2.x. Hadoop 1.x don't have ShutdownHookManager so #1891 doesn't compile on Hadoop 1.x Now, I had a compromised solution for both Hadoop 1.x and 2.x. Only for FileLogger, an unique FileSystem object is created. Author: Kousuke Saruta Closes #1970 from sarutak/SPARK-2970 and squashes the following commits: 240c91e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2970 0e7b45d [Kousuke Saruta] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" e1262ec [Kousuke Saruta] Modified Filelogger to use unique FileSystem instance --- .../scala/org/apache/spark/util/FileLogger.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) 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 2e8fbf5a91ee7..ad8b79af877d8 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -52,7 +52,20 @@ private[spark] class FileLogger( override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } - private val fileSystem = Utils.getHadoopFileSystem(logDir) + /** + * To avoid effects of FileSystem#close or FileSystem.closeAll called from other modules, + * create unique FileSystem instance only for FileLogger + */ + private val fileSystem = { + val conf = SparkHadoopUtil.get.newConfiguration() + val logUri = new URI(logDir) + val scheme = logUri.getScheme + if (scheme == "hdfs") { + conf.setBoolean("fs.hdfs.impl.disable.cache", true) + } + FileSystem.get(logUri, conf) + } + var fileIndex = 0 // Only used if compression is enabled From 25b01fd6bf85ac303094d9bd1d598983461bbe00 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 20 Aug 2014 14:04:39 -0700 Subject: [PATCH 063/192] [SPARK-3149] Connection establishment information is not enough. Author: Kousuke Saruta Closes #2060 from sarutak/SPARK-3149 and squashes the following commits: 1cc89af [Kousuke Saruta] Modified log message of accepting connection (cherry picked from commit c1ba4cd6b4db22a9325eee50dc40a78593a10de1) Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index b3e951ded6e77..e5e1e72cd912b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -418,7 +418,7 @@ private[spark] class ConnectionManager( newConnection.onReceive(receiveMessage) addListeners(newConnection) addConnection(newConnection) - logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]") + logInfo("Accepted connection from [" + newConnection.remoteAddress + "]") } catch { // might happen in case of issues with registering with selector case e: Exception => logError("Error in accept loop", e) From beb705a4723da728be58a08039fb41fa0ffaa4a3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 Aug 2014 15:01:47 -0700 Subject: [PATCH 064/192] [SPARK-2849] Handle driver configs separately in client mode In client deploy mode, the driver is launched from within `SparkSubmit`'s JVM. This means by the time we parse Spark configs from `spark-defaults.conf`, it is already too late to control certain properties of the driver's JVM. We currently ignore these configs in client mode altogether. ``` spark.driver.memory spark.driver.extraJavaOptions spark.driver.extraClassPath spark.driver.extraLibraryPath ``` This PR handles these properties before launching the driver JVM. It achieves this by spawning a separate JVM that runs a new class called `SparkSubmitDriverBootstrapper`, which spawns `SparkSubmit` as a sub-process with the appropriate classpath, library paths, java opts and memory. Author: Andrew Or Closes #1845 from andrewor14/handle-configs-bash and squashes the following commits: bed4bdf [Andrew Or] Change a few comments / messages (minor) 24dba60 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 08fd788 [Andrew Or] Warn against external usages of SparkSubmitDriverBootstrapper ff34728 [Andrew Or] Minor comments 51aeb01 [Andrew Or] Filter out JVM memory in Scala rather than Bash (minor) 9a778f6 [Andrew Or] Fix PySpark: actually kill driver on termination d0f20db [Andrew Or] Don't pass empty library paths, classpath, java opts etc. a78cb26 [Andrew Or] Revert a few changes in utils.sh (minor) 9ba37e2 [Andrew Or] Don't barf when the properties file does not exist 8867a09 [Andrew Or] A few more naming things (minor) 19464ad [Andrew Or] SPARK_SUBMIT_JAVA_OPTS -> SPARK_SUBMIT_OPTS d6488f9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 1ea6bbe [Andrew Or] SparkClassLauncher -> SparkSubmitDriverBootstrapper a91ea19 [Andrew Or] Fix precedence of library paths, classpath, java opts and memory 158f813 [Andrew Or] Remove "client mode" boolean argument c84f5c8 [Andrew Or] Remove debug print statement (minor) b71f52b [Andrew Or] Revert a few more changes (minor) 7d94a8d [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 3a8235d [Andrew Or] Only parse the properties file if special configs exist c37e08d [Andrew Or] Revert a few more changes a396eda [Andrew Or] Nullify my own hard work to simplify bash 0effa1e [Andrew Or] Add code in Scala that handles special configs c886568 [Andrew Or] Fix lines too long + a few comments / style (minor) 7a4190a [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 7396be2 [Andrew Or] Explicitly comment that multi-line properties are not supported fa11ef8 [Andrew Or] Parse the properties file only if the special configs exist 371cac4 [Andrew Or] Add function prefix (minor) be99eb3 [Andrew Or] Fix tests to not include multi-line configs bd0d468 [Andrew Or] Simplify parsing config file by ignoring multi-line arguments 56ac247 [Andrew Or] Use eval and set to simplify splitting 8d4614c [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash aeb79c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 2732ac0 [Andrew Or] Integrate BASH tests into dev/run-tests + log error properly 8d26a5c [Andrew Or] Add tests for bash/utils.sh 4ae24c3 [Andrew Or] Fix bug: escape properly in quote_java_property b3c4cd5 [Andrew Or] Fix bug: count the number of quotes instead of detecting presence c2273fc [Andrew Or] Fix typo (minor) e793e5f [Andrew Or] Handle multi-line arguments 5d8f8c4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra c7b9926 [Andrew Or] Minor changes to spark-defaults.conf.template a992ae2 [Andrew Or] Escape spark.*.extraJavaOptions correctly aabfc7e [Andrew Or] escape -> split (minor) 45a1eb9 [Andrew Or] Fix bug: escape escaped backslashes and quotes properly... 1cdc6b1 [Andrew Or] Fix bug: escape escaped double quotes properly c854859 [Andrew Or] Add small comment c13a2cb [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra 8e552b7 [Andrew Or] Include an example of spark.*.extraJavaOptions de765c9 [Andrew Or] Print spark-class command properly a4df3c4 [Andrew Or] Move parsing and escaping logic to utils.sh dec2343 [Andrew Or] Only export variables if they exist fa2136e [Andrew Or] Escape Java options + parse java properties files properly ef12f74 [Andrew Or] Minor formatting 4ec22a1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra e5cfb46 [Andrew Or] Collapse duplicate code + fix potential whitespace issues 4edcaa8 [Andrew Or] Redirect stdout to stderr for python 130f295 [Andrew Or] Handle spark.driver.memory too 98dd8e3 [Andrew Or] Add warning if properties file does not exist 8843562 [Andrew Or] Fix compilation issues... 75ee6b4 [Andrew Or] Remove accidentally added file 63ed2e9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra 0025474 [Andrew Or] Revert SparkSubmit handling of --driver-* options for only cluster mode a2ab1b0 [Andrew Or] Parse spark.driver.extra* in bash 250cb95 [Andrew Or] Do not ignore spark.driver.extra* for client mode (cherry picked from commit b3ec51bfd795772ff96d18228e979a52ebc82ec4) Signed-off-by: Patrick Wendell --- bin/spark-class | 49 ++++-- bin/spark-submit | 28 +++- bin/utils.sh | 0 conf/spark-defaults.conf.template | 10 +- .../apache/spark/api/python/PythonUtils.scala | 25 --- .../api/python/PythonWorkerFactory.scala | 3 +- .../apache/spark/deploy/PythonRunner.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 17 +- .../SparkSubmitDriverBootstrapper.scala | 149 ++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 21 +++ 10 files changed, 250 insertions(+), 56 deletions(-) mode change 100644 => 100755 bin/utils.sh create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala diff --git a/bin/spark-class b/bin/spark-class index 3f6beca5becf0..22acf92288b3b 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -17,6 +17,8 @@ # limitations under the License. # +# NOTE: Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! + cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -39,7 +41,7 @@ fi if [ -n "$SPARK_MEM" ]; then echo -e "Warning: SPARK_MEM is deprecated, please use a more specific config option" 1>&2 - echo -e "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)." 1>&2 + echo -e "(e.g., spark.executor.memory or spark.driver.memory)." 1>&2 fi # Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -73,11 +75,17 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # 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" + # Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + + # SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} + if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then + OUR_JAVA_OPTS="$OUR_JAVA_OPTS -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" + fi + if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then + OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" + fi ;; *) @@ -101,11 +109,12 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" 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 JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi -export JAVA_OPTS + # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! TOOLS_DIR="$FWDIR"/tools @@ -146,10 +155,28 @@ if $cygwin; then fi export CLASSPATH -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " 1>&2 - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 - echo -e "========================================\n" 1>&2 +# In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. +# Here we must parse the properties file for relevant "spark.driver.*" configs before launching +# the driver JVM itself. Instead of handling this complexity in Bash, we launch a separate JVM +# to prepare the launch environment of this driver JVM. + +if [ -n "$SPARK_SUBMIT_BOOTSTRAP_DRIVER" ]; then + # This is used only if the properties file actually contains these special configs + # Export the environment variables needed by SparkSubmitDriverBootstrapper + export RUNNER + export CLASSPATH + export JAVA_OPTS + export OUR_JAVA_MEM + export SPARK_CLASS=1 + shift # Ignore main class (org.apache.spark.deploy.SparkSubmit) and use our own + exec "$RUNNER" org.apache.spark.deploy.SparkSubmitDriverBootstrapper "$@" +else + # Note: The format of this command is closely echoed in SparkSubmitDriverBootstrapper.scala + if [ -n "$SPARK_PRINT_LAUNCH_COMMAND" ]; then + echo -n "Spark Command: " 1>&2 + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 + echo -e "========================================\n" 1>&2 + fi + exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" fi -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 9e7cecedd0325..32c911cd0438b 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,14 +17,18 @@ # limitations under the License. # +# NOTE: Any changes in this file must be reflected in SparkClassLauncher.scala! + export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=("$@") while (($#)); do if [ "$1" = "--deploy-mode" ]; then - DEPLOY_MODE=$2 + SPARK_SUBMIT_DEPLOY_MODE=$2 + elif [ "$1" = "--properties-file" ]; then + SPARK_SUBMIT_PROPERTIES_FILE=$2 elif [ "$1" = "--driver-memory" ]; then - DRIVER_MEMORY=$2 + export SPARK_SUBMIT_DRIVER_MEMORY=$2 elif [ "$1" = "--driver-library-path" ]; then export SPARK_SUBMIT_LIBRARY_PATH=$2 elif [ "$1" = "--driver-class-path" ]; then @@ -35,10 +39,24 @@ while (($#)); do shift done -DEPLOY_MODE=${DEPLOY_MODE:-"client"} +DEFAULT_PROPERTIES_FILE="$SPARK_HOME/conf/spark-defaults.conf" +export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"} +export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"} + +# For client mode, the driver will be launched in the same JVM that launches +# SparkSubmit, so we may need to read the properties file for any extra class +# paths, library paths, java options and memory early on. Otherwise, it will +# be too late by the time the driver JVM has started. -if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then - export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY +if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FILE" ]]; then + # Parse the properties file only if the special configs exist + contains_special_configs=$( + grep -e "spark.driver.extra*\|spark.driver.memory" "$SPARK_SUBMIT_PROPERTIES_FILE" | \ + grep -v "^[[:space:]]*#" + ) + if [ -n "$contains_special_configs" ]; then + export SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + fi fi exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/bin/utils.sh b/bin/utils.sh old mode 100644 new mode 100755 diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index 2779342769c14..94427029b94d7 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -2,7 +2,9 @@ # This is useful for setting default environmental settings. # Example: -# spark.master spark://master:7077 -# spark.eventLog.enabled true -# spark.eventLog.dir hdfs://namenode:8021/directory -# spark.serializer org.apache.spark.serializer.KryoSerializer +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer +# spark.driver.memory 5g +# spark.executor.extraJavaOptions -XX:+PrintGCDetail -Dkey=value -Dnumbers="one two three" diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 52c70712eea3d..be5ebfa9219d3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -40,28 +40,3 @@ private[spark] object PythonUtils { paths.filter(_ != "").mkString(File.pathSeparator) } } - - -/** - * A utility class to redirect the child process's stdout or stderr. - */ -private[spark] class RedirectThread( - in: InputStream, - out: OutputStream, - name: String) - extends Thread(name) { - - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) - } - } - } -} 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 bf716a8ab025b..4c4796f6c59ba 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 @@ -17,7 +17,6 @@ package org.apache.spark.api.python -import java.lang.Runtime import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} @@ -25,7 +24,7 @@ import scala.collection.mutable import scala.collection.JavaConversions._ import org.apache.spark._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{RedirectThread, Utils} private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 0d6751f3fa6d2..b66c3ba4d5fb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -22,8 +22,8 @@ import java.net.URI import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import org.apache.spark.api.python.{PythonUtils, RedirectThread} -import org.apache.spark.util.Utils +import org.apache.spark.api.python.PythonUtils +import org.apache.spark.util.{RedirectThread, Utils} /** * A main class used by spark-submit to launch Python applications. It executes python as a 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 318509a67a36f..f8cdbc3c392b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -195,18 +195,21 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options - OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraClassPath"), - OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraJavaOptions"), - OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, - sysProp = "spark.files") + sysProp = "spark.files"), + + // Only process driver specific options for cluster mode here, + // because they have already been processed in bash for client mode + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraClassPath"), + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraJavaOptions"), + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraLibraryPath") ) // In client mode, launch the application main class directly diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala new file mode 100644 index 0000000000000..af607e6a4a065 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io.File + +import scala.collection.JavaConversions._ + +import org.apache.spark.util.{RedirectThread, Utils} + +/** + * Launch an application through Spark submit in client mode with the appropriate classpath, + * library paths, java options and memory. These properties of the JVM must be set before the + * driver JVM is launched. The sole purpose of this class is to avoid handling the complexity + * of parsing the properties file for such relevant configs in Bash. + * + * Usage: org.apache.spark.deploy.SparkSubmitDriverBootstrapper + */ +private[spark] object SparkSubmitDriverBootstrapper { + + // Note: This class depends on the behavior of `bin/spark-class` and `bin/spark-submit`. + // Any changes made there must be reflected in this file. + + def main(args: Array[String]): Unit = { + + // This should be called only from `bin/spark-class` + if (!sys.env.contains("SPARK_CLASS")) { + System.err.println("SparkSubmitDriverBootstrapper must be called from `bin/spark-class`!") + System.exit(1) + } + + val submitArgs = args + val runner = sys.env("RUNNER") + val classpath = sys.env("CLASSPATH") + val javaOpts = sys.env("JAVA_OPTS") + val defaultDriverMemory = sys.env("OUR_JAVA_MEM") + + // Spark submit specific environment variables + val deployMode = sys.env("SPARK_SUBMIT_DEPLOY_MODE") + val propertiesFile = sys.env("SPARK_SUBMIT_PROPERTIES_FILE") + val bootstrapDriver = sys.env("SPARK_SUBMIT_BOOTSTRAP_DRIVER") + val submitDriverMemory = sys.env.get("SPARK_SUBMIT_DRIVER_MEMORY") + val submitLibraryPath = sys.env.get("SPARK_SUBMIT_LIBRARY_PATH") + val submitClasspath = sys.env.get("SPARK_SUBMIT_CLASSPATH") + val submitJavaOpts = sys.env.get("SPARK_SUBMIT_OPTS") + + assume(runner != null, "RUNNER must be set") + assume(classpath != null, "CLASSPATH must be set") + assume(javaOpts != null, "JAVA_OPTS must be set") + assume(defaultDriverMemory != null, "OUR_JAVA_MEM must be set") + assume(deployMode == "client", "SPARK_SUBMIT_DEPLOY_MODE must be \"client\"!") + assume(propertiesFile != null, "SPARK_SUBMIT_PROPERTIES_FILE must be set") + assume(bootstrapDriver != null, "SPARK_SUBMIT_BOOTSTRAP_DRIVER must be set") + + // Parse the properties file for the equivalent spark.driver.* configs + val properties = SparkSubmitArguments.getPropertiesFromFile(new File(propertiesFile)).toMap + val confDriverMemory = properties.get("spark.driver.memory") + val confLibraryPath = properties.get("spark.driver.extraLibraryPath") + val confClasspath = properties.get("spark.driver.extraClassPath") + val confJavaOpts = properties.get("spark.driver.extraJavaOptions") + + // Favor Spark submit arguments over the equivalent configs in the properties file. + // Note that we do not actually use the Spark submit values for library path, classpath, + // and Java opts here, because we have already captured them in Bash. + + val newDriverMemory = submitDriverMemory + .orElse(confDriverMemory) + .getOrElse(defaultDriverMemory) + + val newLibraryPath = + if (submitLibraryPath.isDefined) { + // SPARK_SUBMIT_LIBRARY_PATH is already captured in JAVA_OPTS + "" + } else { + confLibraryPath.map("-Djava.library.path=" + _).getOrElse("") + } + + val newClasspath = + if (submitClasspath.isDefined) { + // SPARK_SUBMIT_CLASSPATH is already captured in CLASSPATH + classpath + } else { + classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") + } + + val newJavaOpts = + if (submitJavaOpts.isDefined) { + // SPARK_SUBMIT_OPTS is already captured in JAVA_OPTS + javaOpts + } else { + javaOpts + confJavaOpts.map(" " + _).getOrElse("") + } + + val filteredJavaOpts = Utils.splitCommandString(newJavaOpts) + .filterNot(_.startsWith("-Xms")) + .filterNot(_.startsWith("-Xmx")) + + // Build up command + val command: Seq[String] = + Seq(runner) ++ + Seq("-cp", newClasspath) ++ + Seq(newLibraryPath) ++ + filteredJavaOpts ++ + Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ + Seq("org.apache.spark.deploy.SparkSubmit") ++ + submitArgs + + // Print the launch command. This follows closely the format used in `bin/spark-class`. + if (sys.env.contains("SPARK_PRINT_LAUNCH_COMMAND")) { + System.err.print("Spark Command: ") + System.err.println(command.mkString(" ")) + System.err.println("========================================\n") + } + + // Start the driver JVM + val filteredCommand = command.filter(_.nonEmpty) + val builder = new ProcessBuilder(filteredCommand) + val process = builder.start() + + // Redirect stdin, stdout, and stderr to/from the child JVM + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") + val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") + stdinThread.start() + stdoutThread.start() + stderrThread.start() + + // Terminate on broken pipe, which signals that the parent process has exited. This is + // important for the PySpark shell, where Spark submit itself is a python subprocess. + stdinThread.join() + process.destroy() + } + +} 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 d6d74ce269219..69a84a3604a52 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1480,3 +1480,24 @@ private[spark] object Utils extends Logging { } } + +/** + * A utility class to redirect the child process's stdout or stderr. + */ +private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) + extends Thread(name) { + + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } +} From 311831db71b742a0472d67a1127c818e5ba0a505 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 20 Aug 2014 15:51:14 -0700 Subject: [PATCH 065/192] [SPARK-2967][SQL] Fix sort based shuffle for spark sql. Add explicit row copies when sort based shuffle is on. Author: Michael Armbrust Closes #2066 from marmbrus/sortShuffle and squashes the following commits: fcd7bb2 [Michael Armbrust] Fix sort based shuffle for spark sql. (cherry picked from commit a2e658dcdab614058eefcf50ae2d419ece9b1fe7) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/execution/Exchange.scala | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 77dc2ad733215..09c34b7059fc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree @@ -37,6 +38,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una def output = child.output + /** We must copy rows when sort based shuffle is on */ + protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -45,8 +49,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una @transient val hashExpressions = newMutableProjection(expressions, child.output)() - val mutablePair = new MutablePair[Row, Row]() - iter.map(r => mutablePair.update(hashExpressions(r), r)) + if (sortBasedShuffleOn) { + iter.map(r => (hashExpressions(r), r.copy())) + } else { + val mutablePair = new MutablePair[Row, Row]() + iter.map(r => mutablePair.update(hashExpressions(r), r)) + } } val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) @@ -58,8 +66,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Row, Null](null, null) - iter.map(row => mutablePair.update(row, null)) + if (sortBasedShuffleOn) { + iter.map(row => (row.copy(), null)) + } else { + val mutablePair = new MutablePair[Row, Null](null, null) + iter.map(row => mutablePair.update(row, null)) + } } val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) @@ -69,8 +81,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case SinglePartition => val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Null, Row]() - iter.map(r => mutablePair.update(null, r)) + if (sortBasedShuffleOn) { + iter.map(r => (null, r.copy())) + } else { + val mutablePair = new MutablePair[Null, Row]() + iter.map(r => mutablePair.update(null, r)) + } } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) From 5f72d7bcf553a0216c4849e1918ed74b96d2224a Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 20 Aug 2014 16:00:46 -0700 Subject: [PATCH 066/192] SPARK_LOGFILE and SPARK_ROOT_LOGGER no longer need in spark-daemon.sh Author: wangfei Closes #2057 from scwf/patch-7 and squashes the following commits: 1b7b9a5 [wangfei] SPARK_LOGFILE and SPARK_ROOT_LOGGER no longer need in spark-daemon.sh (cherry picked from commit a1e8b1bc973bc0517681c09e5a5a475c0f395d31) Signed-off-by: Andrew Or --- sbin/spark-daemon.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 323f675b17848..9032f23ea8eff 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -113,8 +113,6 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log -export SPARK_ROOT_LOGGER="INFO,DRFA" log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid From 64e136a640a9ccbde74f7c754b375d175f1991d4 Mon Sep 17 00:00:00 2001 From: Alex Liu Date: Wed, 20 Aug 2014 16:14:06 -0700 Subject: [PATCH 067/192] [SPARK-2846][SQL] Add configureInputJobPropertiesForStorageHandler to initialization of job conf ...al job conf Author: Alex Liu Closes #1927 from alexliu68/SPARK-SQL-2846 and squashes the following commits: e4bdc4c [Alex Liu] SPARK-SQL-2846 add configureInputJobPropertiesForStorageHandler to initial job conf (cherry picked from commit d9e94146a6e65be110a62e3bd0351148912a41d1) Signed-off-by: Michael Armbrust --- .../src/main/scala/org/apache/spark/sql/hive/TableReader.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 82c88280d7754..329f80cad471e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector @@ -249,6 +249,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { FileInputFormat.setInputPaths(jobConf, path) if (tableDesc != null) { + PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc) Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) } val bufferSize = System.getProperty("spark.buffer.size", "65536") From 2c1683efeabe461744509548341b8f93d8b22558 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 13:25:30 -0700 Subject: [PATCH 068/192] [SPARK-2169] Don't copy appName / basePath everywhere. Instead of keeping copies in all pages, just reference the values kept in the base SparkUI instance (by making them available via getters). Author: Marcelo Vanzin Closes #1252 from vanzin/SPARK-2169 and squashes the following commits: 4412fc6 [Marcelo Vanzin] Simplify UIUtils.headerSparkPage signature. 4e5d35a [Marcelo Vanzin] [SPARK-2169] Don't copy appName / basePath everywhere. --- .../apache/spark/deploy/master/Master.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 9 +++++++++ .../scala/org/apache/spark/ui/UIUtils.scala | 12 +++++------- .../scala/org/apache/spark/ui/WebUI.scala | 3 +++ .../apache/spark/ui/env/EnvironmentPage.scala | 4 +--- .../apache/spark/ui/env/EnvironmentTab.scala | 4 +--- .../apache/spark/ui/exec/ExecutorsPage.scala | 5 +---- .../apache/spark/ui/exec/ExecutorsTab.scala | 6 ++---- .../spark/ui/jobs/JobProgressPage.scala | 4 +--- .../apache/spark/ui/jobs/JobProgressTab.scala | 7 +++---- .../org/apache/spark/ui/jobs/PoolPage.scala | 5 +---- .../org/apache/spark/ui/jobs/PoolTable.scala | 7 +++---- .../org/apache/spark/ui/jobs/StagePage.scala | 8 ++------ .../org/apache/spark/ui/jobs/StageTable.scala | 19 ++++++++++--------- .../org/apache/spark/ui/storage/RDDPage.scala | 8 ++------ .../apache/spark/ui/storage/StoragePage.scala | 6 ++---- .../apache/spark/ui/storage/StorageTab.scala | 4 +--- .../spark/streaming/ui/StreamingPage.scala | 3 +-- .../spark/streaming/ui/StreamingTab.scala | 6 ++---- 19 files changed, 51 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index cfa2c028a807b..5017273e87c07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -697,7 +697,7 @@ private[spark] class Master( appIdToUI(app.id) = ui webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it - app.desc.appUiUrl = ui.basePath + app.desc.appUiUrl = ui.getBasePath true } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 6c788a37dc70b..cccd59d122a92 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -76,6 +76,8 @@ private[spark] class SparkUI( } } + def getAppName = appName + /** Set the app name for this UI. */ def setAppName(name: String) { appName = name @@ -100,6 +102,13 @@ private[spark] class SparkUI( private[spark] def appUIAddress = s"http://$appUIHostPort" } +private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) + extends WebUITab(parent, prefix) { + + def appName: String = parent.getAppName + +} + private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 715cc2f4df8dd..bee6dad3387e5 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -163,17 +163,15 @@ private[spark] object UIUtils extends Logging { /** Returns a spark page with correctly formatted headers */ def headerSparkPage( - content: => Seq[Node], - basePath: String, - appName: String, title: String, - tabs: Seq[WebUITab], - activeTab: WebUITab, + content: => Seq[Node], + activeTab: SparkUITab, refreshInterval: Option[Int] = None): Seq[Node] = { - val header = tabs.map { tab => + val appName = activeTab.appName + val header = activeTab.headerTabs.map { tab =>
  • - {tab.name} + {tab.name}
  • } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 5f52f95088007..5d88ca403a674 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -50,6 +50,7 @@ private[spark] abstract class WebUI( protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) + def getBasePath: String = basePath def getTabs: Seq[WebUITab] = tabs.toSeq def getHandlers: Seq[ServletContextHandler] = handlers.toSeq def getSecurityManager: SecurityManager = securityManager @@ -135,6 +136,8 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { /** Get a list of header tabs from the parent UI. */ def headerTabs: Seq[WebUITab] = parent.getTabs + + def basePath: String = parent.getBasePath } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index b347eb1b83c1f..f0a1174a71d34 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -24,8 +24,6 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -45,7 +43,7 @@ private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("")

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) + UIUtils.headerSparkPage("Environment", content, parent) } private def propertyHeader = Seq("Name", "Value") diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index bbbe55ecf44a1..0d158fbe638d3 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -21,9 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.ui._ -private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { - val appName = parent.appName - val basePath = parent.basePath +private[ui] class EnvironmentTab(parent: SparkUI) extends SparkUITab(parent, "environment") { val listener = new EnvironmentListener attachPage(new EnvironmentPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index b814b0e6b8509..02df4e8fe61af 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -43,8 +43,6 @@ private case class ExecutorSummaryInfo( maxMemory: Long) private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -101,8 +99,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { ; - UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", - parent.headerTabs, parent) + UIUtils.headerSparkPage("Executors (" + execInfo.size + ")", content, parent) } /** Render an HTML row representing an executor */ diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 5c2d1d1fe75d3..61eb111cd9100 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -23,11 +23,9 @@ import org.apache.spark.ExceptionFailure import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.{SparkUI, WebUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab} -private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { - val appName = parent.appName - val basePath = parent.basePath +private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { val listener = new ExecutorsListener(parent.storageStatusListener) attachPage(new ExecutorsPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 0da62892118d4..a82f71ed08475 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -26,8 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private val listener = parent.listener @@ -94,7 +92,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) + UIUtils.headerSparkPage("Spark Stages", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 8a01ec80c9dd6..c16542c9db30f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -21,12 +21,10 @@ import javax.servlet.http.HttpServletRequest import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, WebUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { - val appName = parent.appName - val basePath = parent.basePath +private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val live = parent.live val sc = parent.sc val conf = if (live) sc.conf else new SparkConf @@ -53,4 +51,5 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag Thread.sleep(100) } } + } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 0a2bf31833d2b..7a6c7d1a497ed 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,8 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { - private val appName = parent.appName - private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private val listener = parent.listener @@ -51,8 +49,7 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, - parent.headerTabs, parent) + UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index f4b68f241966d..64178e1e33d41 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -25,7 +25,6 @@ import org.apache.spark.ui.UIUtils /** Table showing list of pools */ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { - private val basePath = parent.basePath private val listener = parent.listener def toNodeSeq: Seq[Node] = { @@ -59,11 +58,11 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { case Some(stages) => stages.size case None => 0 } + val href = "%s/stages/pool?poolname=%s" + .format(UIUtils.prependBaseUri(parent.basePath), p.name) - - {p.name} - + {p.name} {p.minShare} {p.weight} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 8bc1ba758cf77..d4eb02722ad12 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -29,8 +29,6 @@ import org.apache.spark.scheduler.AccumulableInfo /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -44,8 +42,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet - return UIUtils.headerSparkPage(content, basePath, appName, - "Details for Stage %s".format(stageId), parent.headerTabs, parent) + return UIUtils.headerSparkPage("Details for Stage %s".format(stageId), content, parent) } val stageData = stageDataOption.get @@ -227,8 +224,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { maybeAccumulableTable ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), - parent.headerTabs, parent) + UIUtils.headerSparkPage("Details for Stage %d".format(stageId), content, parent) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 15998404ed612..16ad0df45aa0d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -32,7 +32,6 @@ private[ui] class StageTableBase( parent: JobProgressTab, killEnabled: Boolean = false) { - private val basePath = parent.basePath private val listener = parent.listener protected def isFairScheduler = parent.isFairScheduler @@ -88,17 +87,19 @@ private[ui] class StageTableBase( private def makeDescription(s: StageInfo): Seq[Node] = { // scalastyle:off val killLink = if (killEnabled) { + val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" + .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" + .format(s.stageId) - (kill) + (kill) } // scalastyle:on - val nameLink = - - {s.name} - + val nameLinkUri ="%s/stages/stage?id=%s" + .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) val details = if (s.details.nonEmpty) { @@ -111,7 +112,7 @@ private[ui] class StageTableBase( Text("RDD: ") ++ // scalastyle:off cachedRddInfos.map { i => - {i.name} + {i.name} } // scalastyle:on }} @@ -157,7 +158,7 @@ private[ui] class StageTableBase( {if (isFairScheduler) { + .format(UIUtils.prependBaseUri(parent.basePath), stageData.schedulingPool)}> {stageData.schedulingPool} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 84ac53da47552..8a0075ae8daf7 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -27,8 +27,6 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -36,8 +34,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", - parent.headerTabs, parent) + return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) } // Worker table @@ -96,8 +93,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { ; - UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, - parent.headerTabs, parent) + UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 9813d9330ac7f..716591c9ed449 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -27,14 +27,12 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) + UIUtils.headerSparkPage("Storage", content, parent) } /** Header fields for the RDD table */ @@ -52,7 +50,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { // scalastyle:off - + {rdd.name} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 5f6740d495521..67f72a94f0269 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -25,9 +25,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { - val appName = parent.appName - val basePath = parent.basePath +private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { val listener = new StorageListener(parent.storageStatusListener) attachPage(new StoragePage(this)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 451b23e01c995..1353e487c72cf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -42,8 +42,7 @@ private[ui] class StreamingPage(parent: StreamingTab)

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateReceiverStats() ++ generateBatchStatsTable() - UIUtils.headerSparkPage( - content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) + UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } /** Generate basic stats of the streaming program */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 51448d15c6516..34ac254f337eb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -19,15 +19,13 @@ package org.apache.spark.streaming.ui import org.apache.spark.Logging import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.WebUITab +import org.apache.spark.ui.SparkUITab /** Spark Web UI tab that shows statistics of a streaming job */ private[spark] class StreamingTab(ssc: StreamingContext) - extends WebUITab(ssc.sc.ui, "streaming") with Logging { + extends SparkUITab(ssc.sc.ui, "streaming") with Logging { val parent = ssc.sc.ui - val appName = parent.appName - val basePath = parent.basePath val listener = new StreamingJobProgressListener(ssc) ssc.addStreamingListener(listener) From dc05282bafce8e11de35d7d2f489a8b50a91661d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 20 Aug 2014 15:37:27 -0700 Subject: [PATCH 069/192] [SPARK-2298] Encode stage attempt in SparkListener & UI. Simple way to reproduce this in the UI: ```scala val f = new java.io.File("/tmp/test") f.delete() sc.parallelize(1 to 2, 2).map(x => (x,x )).repartition(3).mapPartitionsWithContext { case (context, iter) => if (context.partitionId == 0) { val f = new java.io.File("/tmp/test") if (!f.exists) { f.mkdir() System.exit(0); } } iter }.count() ``` Author: Reynold Xin Closes #1545 from rxin/stage-attempt and squashes the following commits: 3ee1d2a [Reynold Xin] - Rename attempt to retry in UI. - Properly report stage failure in FetchFailed. 40a6bd5 [Reynold Xin] Updated test suites. c414c36 [Reynold Xin] Fixed the hanging in JobCancellationSuite. b3e2eed [Reynold Xin] Oops previous code didn't compile. 0f36075 [Reynold Xin] Mark unknown stage attempt with id -1 and drop that in JobProgressListener. 6c08b07 [Reynold Xin] Addressed code review feedback. 4e5faa2 [Reynold Xin] [SPARK-2298] Encode stage attempt in SparkListener & UI. --- .../apache/spark/scheduler/DAGScheduler.scala | 77 +-- .../spark/scheduler/SparkListener.scala | 11 +- .../org/apache/spark/scheduler/Stage.scala | 8 +- .../apache/spark/scheduler/StageInfo.scala | 11 +- .../spark/scheduler/TaskSchedulerImpl.scala | 8 +- .../org/apache/spark/scheduler/TaskSet.scala | 4 - .../apache/spark/ui/jobs/ExecutorTable.scala | 6 +- .../spark/ui/jobs/JobProgressListener.scala | 40 +- .../org/apache/spark/ui/jobs/StagePage.scala | 11 +- .../org/apache/spark/ui/jobs/StageTable.scala | 14 +- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../storage/StorageStatusListenerSuite.scala | 17 +- .../ui/jobs/JobProgressListenerSuite.scala | 68 +-- .../spark/ui/storage/StorageTabSuite.scala | 16 +- .../apache/spark/util/JsonProtocolSuite.scala | 476 ++++++++++++++---- 15 files changed, 555 insertions(+), 224 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b86cfbfa48fbe..34131984570e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -164,7 +164,7 @@ class DAGScheduler( */ def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, Int, TaskMetrics)], // (taskId, stageId, metrics) + taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) blockManagerId: BlockManagerId): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) implicit val timeout = Timeout(600 seconds) @@ -677,7 +677,10 @@ class DAGScheduler( } private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + // Note that there is a chance that this task is launched after the stage is cancelled. + // In that case, we wouldn't have the stage anymore in stageIdToStage. + val stageAttemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) + listenerBus.post(SparkListenerTaskStart(task.stageId, stageAttemptId, taskInfo)) submitWaitingStages() } @@ -695,8 +698,8 @@ class DAGScheduler( // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" runningStages.foreach { stage => - stage.info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + stage.latestInfo.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } @@ -781,7 +784,16 @@ class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() - var tasks = ArrayBuffer[Task[_]]() + + // First figure out the indexes of partition ids to compute. + val partitionsToCompute: Seq[Int] = { + if (stage.isShuffleMap) { + (0 until stage.numPartitions).filter(id => stage.outputLocs(id) == Nil) + } else { + val job = stage.resultOfJob.get + (0 until job.numPartitions).filter(id => !job.finished(id)) + } + } val properties = if (jobIdToActiveJob.contains(jobId)) { jobIdToActiveJob(stage.jobId).properties @@ -795,7 +807,8 @@ class DAGScheduler( // serializable. If tasks are not serializable, a SparkListenerStageCompleted event // will be posted, which should always come after a corresponding SparkListenerStageSubmitted // event. - listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) + stage.latestInfo = StageInfo.fromStage(stage, Some(partitionsToCompute.size)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast @@ -826,20 +839,19 @@ class DAGScheduler( return } - if (stage.isShuffleMap) { - for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { - val locs = getPreferredLocs(stage.rdd, p) - val part = stage.rdd.partitions(p) - tasks += new ShuffleMapTask(stage.id, taskBinary, part, locs) + val tasks: Seq[Task[_]] = if (stage.isShuffleMap) { + partitionsToCompute.map { id => + val locs = getPreferredLocs(stage.rdd, id) + val part = stage.rdd.partitions(id) + new ShuffleMapTask(stage.id, taskBinary, part, locs) } } else { - // This is a final stage; figure out its job's missing partitions val job = stage.resultOfJob.get - for (id <- 0 until job.numPartitions if !job.finished(id)) { + partitionsToCompute.map { id => val p: Int = job.partitions(id) val part = stage.rdd.partitions(p) val locs = getPreferredLocs(stage.rdd, p) - tasks += new ResultTask(stage.id, taskBinary, part, locs, id) + new ResultTask(stage.id, taskBinary, part, locs, id) } } @@ -869,11 +881,11 @@ class DAGScheduler( logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stage.info.submissionTime = Some(clock.getTime()) + stage.latestInfo.submissionTime = Some(clock.getTime()) } else { // Because we posted SparkListenerStageSubmitted earlier, we should post // SparkListenerStageCompleted here in case there are no tasks to run. - listenerBus.post(SparkListenerStageCompleted(stage.info)) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) runningStages -= stage @@ -892,8 +904,9 @@ class DAGScheduler( // The success case is dealt with separately below, since we need to compute accumulator // updates before posting. if (event.reason != Success) { - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + val attemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) + listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason, + event.taskInfo, event.taskMetrics)) } if (!stageIdToStage.contains(task.stageId)) { @@ -902,14 +915,19 @@ class DAGScheduler( } val stage = stageIdToStage(task.stageId) - def markStageAsFinished(stage: Stage) = { - val serviceTime = stage.info.submissionTime match { + def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { + val serviceTime = stage.latestInfo.submissionTime match { case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } - logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stage.info.completionTime = Some(clock.getTime()) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + if (errorMessage.isEmpty) { + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) + stage.latestInfo.completionTime = Some(clock.getTime()) + } else { + stage.latestInfo.stageFailed(errorMessage.get) + logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + } + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) runningStages -= stage } event.reason match { @@ -924,7 +942,7 @@ class DAGScheduler( val name = acc.name.get val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) val stringValue = Accumulators.stringifyValue(acc.value) - stage.info.accumulables(id) = AccumulableInfo(id, name, stringValue) + stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) event.taskInfo.accumulables += AccumulableInfo(id, name, Some(stringPartialValue), stringValue) } @@ -935,8 +953,8 @@ class DAGScheduler( logError(s"Failed to update accumulators for $task", e) } } - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, + event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => @@ -1029,6 +1047,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) + markStageAsFinished(failedStage, Some("Fetch failure")) runningStages -= failedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + failedStage + " (" + failedStage.name + @@ -1142,7 +1161,7 @@ class DAGScheduler( } val dependentJobs: Seq[ActiveJob] = activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq - failedStage.info.completionTime = Some(clock.getTime()) + failedStage.latestInfo.completionTime = Some(clock.getTime()) for (job <- dependentJobs) { failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } @@ -1182,8 +1201,8 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - stage.info.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + stage.latestInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d01d318633877..86ca8445a1124 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -39,7 +39,8 @@ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Propert case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent @DeveloperApi -case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: TaskInfo) + extends SparkListenerEvent @DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent @@ -47,6 +48,7 @@ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListe @DeveloperApi case class SparkListenerTaskEnd( stageId: Int, + stageAttemptId: Int, taskType: String, reason: TaskEndReason, taskInfo: TaskInfo, @@ -75,10 +77,15 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +/** + * Periodic updates from executors. + * @param execId executor id + * @param taskMetrics sequence of (task id, stage id, stage attempt, metrics) + */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - taskMetrics: Seq[(Long, Int, TaskMetrics)]) + taskMetrics: Seq[(Long, Int, Int, TaskMetrics)]) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 800905413d145..071568cdfb429 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -43,6 +43,9 @@ import org.apache.spark.util.CallSite * stage, the callSite gives the user code that created the RDD being shuffled. For a result * stage, the callSite gives the user code that executes the associated action (e.g. count()). * + * A single stage can consist of multiple attempts. In that case, the latestInfo field will + * be updated for each attempt. + * */ private[spark] class Stage( val id: Int, @@ -71,8 +74,8 @@ private[spark] class Stage( val name = callSite.shortForm val details = callSite.longForm - /** Pointer to the [StageInfo] object, set by DAGScheduler. */ - var info: StageInfo = StageInfo.fromStage(this) + /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ + var latestInfo: StageInfo = StageInfo.fromStage(this) def isAvailable: Boolean = { if (!isShuffleMap) { @@ -116,6 +119,7 @@ private[spark] class Stage( } } + /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { val id = nextAttemptId nextAttemptId += 1 diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 2a407e47a05bd..c6dc3369ba5cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.storage.RDDInfo @DeveloperApi class StageInfo( val stageId: Int, + val attemptId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo], @@ -56,9 +57,15 @@ private[spark] object StageInfo { * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a * sequence of narrow dependencies should also be associated with this Stage. */ - def fromStage(stage: Stage): StageInfo = { + def fromStage(stage: Stage, numTasks: Option[Int] = None): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos, stage.details) + new StageInfo( + stage.id, + stage.attemptId, + stage.name, + numTasks.getOrElse(stage.numTasks), + rddInfos, + stage.details) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6c0d1b2752a81..ad051e59af86d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -333,12 +333,12 @@ private[spark] class TaskSchedulerImpl( execId: String, taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId): Boolean = { - val metricsWithStageIds = taskMetrics.flatMap { - case (id, metrics) => { + + val metricsWithStageIds: Array[(Long, Int, Int, TaskMetrics)] = synchronized { + taskMetrics.flatMap { case (id, metrics) => taskIdToTaskSetId.get(id) .flatMap(activeTaskSets.get) - .map(_.stageId) - .map(x => (id, x, metrics)) + .map(taskSetMgr => (id, taskSetMgr.stageId, taskSetMgr.taskSet.attempt, metrics)) } } dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 613fa7850bb25..c3ad325156f53 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,9 +31,5 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill(interruptThread: Boolean) { - tasks.foreach(_.kill(interruptThread)) - } - override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 0cc51c873727d..2987dc04494a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -24,8 +24,8 @@ import org.apache.spark.ui.{ToolTips, UIUtils} import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils -/** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { +/** Stage summary grouped by executors. */ +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobProgressTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { @@ -65,7 +65,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { executorIdToAddress.put(executorId, address) } - listener.stageIdToData.get(stageId) match { + listener.stageIdToData.get((stageId, stageAttemptId)) match { case Some(stageData: StageUIData) => stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 74cd637d88155..f7f918fd521a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -43,12 +43,16 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // How many stages to remember val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) - val activeStages = HashMap[Int, StageInfo]() + // Map from stageId to StageInfo + val activeStages = new HashMap[Int, StageInfo] + + // Map from (stageId, attemptId) to StageUIData + val stageIdToData = new HashMap[(Int, Int), StageUIData] + val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() - val stageIdToData = new HashMap[Int, StageUIData] - + // Map from pool name to a hash map (map from stage id to StageInfo). val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() @@ -59,9 +63,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo - val stageId = stage.stageId - val stageData = stageIdToData.getOrElseUpdate(stageId, { - logWarning("Stage completed for unknown stage " + stageId) + val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { + logWarning("Stage completed for unknown stage " + stage.stageId) new StageUIData }) @@ -69,8 +72,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.accumulables(id) = info } - poolToActiveStages.get(stageData.schedulingPool).foreach(_.remove(stageId)) - activeStages.remove(stageId) + poolToActiveStages.get(stageData.schedulingPool).foreach { hashMap => + hashMap.remove(stage.stageId) + } + activeStages.remove(stage.stageId) if (stage.failureReason.isEmpty) { completedStages += stage trimIfNecessary(completedStages) @@ -84,7 +89,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) - stages.take(toRemove).foreach { s => stageIdToData.remove(s.stageId) } + stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) } stages.trimStart(toRemove) } } @@ -98,21 +103,21 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) - val stageData = stageIdToData.getOrElseUpdate(stage.stageId, new StageUIData) + val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) stageData.schedulingPool = poolName stageData.description = Option(stageSubmitted.properties).flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) stages(stage.stageId) = stage } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val taskInfo = taskStart.taskInfo if (taskInfo != null) { - val stageData = stageIdToData.getOrElseUpdate(taskStart.stageId, { + val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { logWarning("Task start for unknown stage " + taskStart.stageId) new StageUIData }) @@ -128,8 +133,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo - if (info != null) { - val stageData = stageIdToData.getOrElseUpdate(taskEnd.stageId, { + // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task + // compeletion event is for. Let's just drop it here. This means we might have some speculation + // tasks on the web ui that's never marked as complete. + if (info != null && taskEnd.stageAttemptId != -1) { + val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), { logWarning("Task end for unknown stage " + taskEnd.stageId) new StageUIData }) @@ -222,8 +230,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - for ((taskId, sid, taskMetrics) <- executorMetricsUpdate.taskMetrics) { - val stageData = stageIdToData.getOrElseUpdate(sid, { + for ((taskId, sid, sAttempt, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + val stageData = stageIdToData.getOrElseUpdate((sid, sAttempt), { logWarning("Metrics update for task in unknown stage " + sid) new StageUIData }) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d4eb02722ad12..db01be596e073 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -34,7 +34,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt - val stageDataOption = listener.stageIdToData.get(stageId) + val stageAttemptId = request.getParameter("attempt").toInt + val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId)) if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { val content = @@ -42,14 +43,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet - return UIUtils.headerSparkPage("Details for Stage %s".format(stageId), content, parent) + return UIUtils.headerSparkPage( + s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent) } val stageData = stageDataOption.get val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) - val accumulables = listener.stageIdToData(stageId).accumulables + val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables val hasInput = stageData.inputBytes > 0 val hasShuffleRead = stageData.shuffleReadBytes > 0 val hasShuffleWrite = stageData.shuffleWriteBytes > 0 @@ -211,7 +213,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def quantileRow(data: Seq[Node]): Seq[Node] = {data} Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } - val executorTable = new ExecutorTable(stageId, parent) + + val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) val maybeAccumulableTable: Seq[Node] = if (accumulables.size > 0) {

    Accumulators

    ++ accumulableTable } else Seq() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 16ad0df45aa0d..2e67310594784 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -97,8 +97,8 @@ private[ui] class StageTableBase( } // scalastyle:on - val nameLinkUri ="%s/stages/stage?id=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" + .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -121,7 +121,7 @@ private[ui] class StageTableBase( } val stageDesc = for { - stageData <- listener.stageIdToData.get(s.stageId) + stageData <- listener.stageIdToData.get((s.stageId, s.attemptId)) desc <- stageData.description } yield {
    {desc}
    @@ -131,7 +131,7 @@ private[ui] class StageTableBase( } protected def stageRow(s: StageInfo): Seq[Node] = { - val stageDataOption = listener.stageIdToData.get(s.stageId) + val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) if (stageDataOption.isEmpty) { return {s.stageId}No data available for this stage } @@ -154,7 +154,11 @@ private[ui] class StageTableBase( val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" - {s.stageId} ++ + {if (s.attemptId > 0) { + {s.stageId} (retry {s.attemptId}) + } else { + {s.stageId} + }} ++ {if (isFairScheduler) { Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ + ("Stage Attempt ID" -> taskStart.stageAttemptId) ~ ("Task Info" -> taskInfoToJson(taskInfo)) } @@ -112,6 +113,7 @@ private[spark] object JsonProtocol { val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ + ("Stage Attempt ID" -> taskEnd.stageAttemptId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ ("Task Info" -> taskInfoToJson(taskInfo)) ~ @@ -187,6 +189,7 @@ private[spark] object JsonProtocol { val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ + ("Stage Attempt ID" -> stageInfo.attemptId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ @@ -419,8 +422,9 @@ private[spark] object JsonProtocol { def taskStartFromJson(json: JValue): SparkListenerTaskStart = { val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val taskInfo = taskInfoFromJson(json \ "Task Info") - SparkListenerTaskStart(stageId, taskInfo) + SparkListenerTaskStart(stageId, stageAttemptId, taskInfo) } def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { @@ -430,11 +434,12 @@ private[spark] object JsonProtocol { def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val taskType = (json \ "Task Type").extract[String] val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") val taskInfo = taskInfoFromJson(json \ "Task Info") val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") - SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + SparkListenerTaskEnd(stageId, stageAttemptId, taskType, taskEndReason, taskInfo, taskMetrics) } def jobStartFromJson(json: JValue): SparkListenerJobStart = { @@ -492,6 +497,7 @@ private[spark] object JsonProtocol { def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] + val attemptId = (json \ "Attempt ID").extractOpt[Int].getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_)) @@ -504,7 +510,7 @@ private[spark] object JsonProtocol { case None => Seq[AccumulableInfo]() } - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos, details) + val stageInfo = new StageInfo(stageId, attemptId, stageName, numTasks, rddInfos, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 51fb646a3cb61..7671cb969a26b 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -69,10 +69,10 @@ class StorageStatusListenerSuite extends FunSuite { // Task end with no updated blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } @@ -92,13 +92,13 @@ class StorageStatusListenerSuite extends FunSuite { // Task end with new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -111,13 +111,14 @@ class StorageStatusListenerSuite extends FunSuite { val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -135,8 +136,8 @@ class StorageStatusListenerSuite extends FunSuite { val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) taskMetrics2.updatedBlocks = Some(Seq(block3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 147ec0bc52e39..3370dd4156c3f 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -34,12 +34,12 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val listener = new JobProgressListener(conf) def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") SparkListenerStageSubmitted(stageInfo) } def createStageEndEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") SparkListenerStageCompleted(stageInfo) } @@ -70,33 +70,37 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskInfo.finishTime = 1 var task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) - .shuffleRead === 1000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.size === 1) // finish this task, should get updated duration taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) - .shuffleRead === 2000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 2000) // finish this task, should get updated duration taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-2", fail()) - .shuffleRead === 1000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-2", fail()).shuffleRead === 1000) } test("test task success vs failure counting for different task end reasons") { @@ -119,16 +123,18 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc UnknownReason) var failCount = 0 for (reason <- taskFailedReasons) { - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, reason, taskInfo, metrics)) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, reason, taskInfo, metrics)) failCount += 1 - assert(listener.stageIdToData(task.stageId).numCompleteTasks === 0) - assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) + assert(listener.stageIdToData((task.stageId, 0)).numCompleteTasks === 0) + assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) } // Make sure we count success as success. - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, metrics)) - assert(listener.stageIdToData(task.stageId).numCompleteTasks === 1) - assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 1, taskType, Success, taskInfo, metrics)) + assert(listener.stageIdToData((task.stageId, 1)).numCompleteTasks === 1) + assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) } test("test update metrics") { @@ -163,18 +169,18 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskInfo } - listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1234L))) - listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1235L))) - listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1236L))) - listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1237L))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1234L))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1235L))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1236L))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1237L))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( - (1234L, 0, makeTaskMetrics(0)), - (1235L, 0, makeTaskMetrics(100)), - (1236L, 1, makeTaskMetrics(200))))) + (1234L, 0, 0, makeTaskMetrics(0)), + (1235L, 0, 0, makeTaskMetrics(100)), + (1236L, 1, 0, makeTaskMetrics(200))))) - var stage0Data = listener.stageIdToData.get(0).get - var stage1Data = listener.stageIdToData.get(1).get + var stage0Data = listener.stageIdToData.get((0, 0)).get + var stage1Data = listener.stageIdToData.get((1, 0)).get assert(stage0Data.shuffleReadBytes == 102) assert(stage1Data.shuffleReadBytes == 201) assert(stage0Data.shuffleWriteBytes == 106) @@ -195,14 +201,14 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc .totalBlocksFetched == 202) // task that was included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(0, taskType, Success, makeTaskInfo(1234L, 1), + listener.onTaskEnd(SparkListenerTaskEnd(0, 0, taskType, Success, makeTaskInfo(1234L, 1), makeTaskMetrics(300))) // task that wasn't included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(1, taskType, Success, makeTaskInfo(1237L, 1), + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, taskType, Success, makeTaskInfo(1237L, 1), makeTaskMetrics(400))) - stage0Data = listener.stageIdToData.get(0).get - stage1Data = listener.stageIdToData.get(1).get + stage0Data = listener.stageIdToData.get((0, 0)).get + stage1Data = listener.stageIdToData.get((1, 0)).get assert(stage0Data.shuffleReadBytes == 402) assert(stage1Data.shuffleReadBytes == 602) assert(stage0Data.shuffleWriteBytes == 406) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 6e68dcb3425aa..b860177705d84 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -53,7 +53,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.isEmpty) // 2 RDDs are known, but none are cached - val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0, rddInfo1), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.isEmpty) @@ -63,7 +63,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo3Cached = rddInfo3 rddInfo2Cached.numCachedPartitions = 1 rddInfo3Cached.numCachedPartitions = 1 - val stageInfo1 = new StageInfo(1, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + val stageInfo1 = new StageInfo(1, 0, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) @@ -71,7 +71,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { // Submitting RDDInfos with duplicate IDs does nothing val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) rddInfo0Cached.numCachedPartitions = 1 - val stageInfo0Cached = new StageInfo(0, "0", 100, Seq(rddInfo0), "details") + val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) @@ -87,7 +87,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo1Cached = rddInfo1 rddInfo0Cached.numCachedPartitions = 1 rddInfo1Cached.numCachedPartitions = 1 - val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.size === 2) @@ -106,7 +106,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo(0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) @@ -116,7 +116,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(!storageListener._rddInfoMap(2).isCached) // Task end with no updated blocks. This should not change anything. - bus.postToAll(SparkListenerTaskEnd(0, "obliteration", Success, taskInfo, new TaskMetrics)) + bus.postToAll(SparkListenerTaskEnd(0, 0, "obliteration", Success, taskInfo, new TaskMetrics)) assert(storageListener._rddInfoMap.size === 3) assert(storageListener.rddInfoList.size === 0) @@ -128,7 +128,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L, 0L)) )) - bus.postToAll(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo, metrics1)) + bus.postToAll(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo, metrics1)) assert(storageListener._rddInfoMap(0).memSize === 800L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).tachyonSize === 200L) @@ -150,7 +150,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L, 0L)) // doesn't actually exist )) - bus.postToAll(SparkListenerTaskEnd(2, "obliteration", Success, taskInfo, metrics2)) + bus.postToAll(SparkListenerTaskEnd(2, 0, "obliteration", Success, taskInfo, metrics2)) assert(storageListener._rddInfoMap(0).memSize === 400L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).tachyonSize === 200L) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 97ffb07662482..2fd3b9cfd221a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -35,13 +35,13 @@ class JsonProtocolSuite extends FunSuite { val stageSubmitted = SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) - val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 1, 444L, false)) + val taskStart = SparkListenerTaskStart(111, 0, makeTaskInfo(222L, 333, 1, 444L, false)) val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true)) - val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + val taskEnd = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = false)) - val taskEndWithHadoopInput = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + val taskEndWithHadoopInput = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true)) val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) @@ -397,7 +397,8 @@ class JsonProtocolSuite extends FunSuite { private def assertJsonStringEquals(json1: String, json2: String) { val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") - assert(formatJsonString(json1) === formatJsonString(json2)) + assert(formatJsonString(json1) === formatJsonString(json2), + s"input ${formatJsonString(json1)} got ${formatJsonString(json2)}") } private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { @@ -485,7 +486,7 @@ class JsonProtocolSuite extends FunSuite { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, "greetings", b, rddInfos, "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, "details") val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 @@ -558,84 +559,246 @@ class JsonProtocolSuite extends FunSuite { private val stageSubmittedJsonString = """ - {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": - "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details", - "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]},"Properties": - {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + |{ + | "Event": "SparkListenerStageSubmitted", + | "Stage Info": { + | "Stage ID": 100, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} """ private val stageCompletedJsonString = """ - {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": - "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, - "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, - "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details", - "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]}} + |{ + | "Event": "SparkListenerStageCompleted", + | "Stage Info": { + | "Stage ID": 101, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 201, + | "RDD Info": [ + | { + | "RDD ID": 101, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 201, + | "Number of Cached Partitions": 301, + | "Memory Size": 401, + | "Tachyon Size": 0, + | "Disk Size": 501 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + |} """ private val taskStartJsonString = """ - |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, - |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", - |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0, - |"Failed":false,"Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - |"Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - |{"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}]}} + |{ + | "Event": "SparkListenerTaskStart", + | "Stage ID": 111, + | "Stage Attempt ID": 0, + | "Task Info": { + | "Task ID": 222, + | "Index": 333, + | "Attempt": 1, + | "Launch Time": 444, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] + | } + |} """.stripMargin private val taskGettingResultJsonString = """ - |{"Event":"SparkListenerTaskGettingResult","Task Info": - | {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0, - | "Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] + |{ + | "Event": "SparkListenerTaskGettingResult", + | "Task Info": { + | "Task ID": 1000, + | "Index": 2000, + | "Attempt": 5, + | "Launch Time": 3000, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": true, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] | } |} """.stripMargin private val taskEndJsonString = """ - |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", - |"Task End Reason":{"Reason":"Success"}, - |"Task Info":{ - | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] - |}, - |"Task Metrics":{ - | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, - | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, - | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, - | "Shuffle Read Metrics":{ - | "Shuffle Finish Time":900, - | "Remote Blocks Fetched":800, - | "Local Blocks Fetched":700, - | "Fetch Wait Time":900, - | "Remote Bytes Read":1000 + |{ + | "Event": "SparkListenerTaskEnd", + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Task Type": "ShuffleMapTask", + | "Task End Reason": { + | "Reason": "Success" | }, - | "Shuffle Write Metrics":{ - | "Shuffle Bytes Written":1200, - | "Shuffle Write Time":1500 + | "Task Info": { + | "Task ID": 123, + | "Index": 234, + | "Attempt": 67, + | "Launch Time": 345, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] | }, - | "Updated Blocks":[ - | {"Block ID":"rdd_0_0", - | "Status":{ - | "Storage Level":{ - | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - | "Replication":2 - | }, - | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + | "Task Metrics": { + | "Host Name": "localhost", + | "Executor Deserialize Time": 300, + | "Executor Run Time": 400, + | "Result Size": 500, + | "JVM GC Time": 600, + | "Result Serialization Time": 700, + | "Memory Bytes Spilled": 800, + | "Disk Bytes Spilled": 0, + | "Shuffle Read Metrics": { + | "Shuffle Finish Time": 900, + | "Remote Blocks Fetched": 800, + | "Local Blocks Fetched": 700, + | "Fetch Wait Time": 900, + | "Remote Bytes Read": 1000 + | }, + | "Shuffle Write Metrics": { + | "Shuffle Bytes Written": 1200, + | "Shuffle Write Time": 1500 + | }, + | "Updated Blocks": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Tachyon Size": 0, + | "Disk Size": 0 + | } | } - | } | ] | } |} @@ -643,80 +806,187 @@ class JsonProtocolSuite extends FunSuite { private val taskEndWithHadoopInputJsonString = """ - |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", - |"Task End Reason":{"Reason":"Success"}, - |"Task Info":{ - | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] - |}, - |"Task Metrics":{ - | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, - | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, - | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, - | "Shuffle Write Metrics":{"Shuffle Bytes Written":1200,"Shuffle Write Time":1500}, - | "Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":2100}, - | "Updated Blocks":[ - | {"Block ID":"rdd_0_0", - | "Status":{ - | "Storage Level":{ - | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - | "Replication":2 - | }, - | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + |{ + | "Event": "SparkListenerTaskEnd", + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Task Type": "ShuffleMapTask", + | "Task End Reason": { + | "Reason": "Success" + | }, + | "Task Info": { + | "Task ID": 123, + | "Index": 234, + | "Attempt": 67, + | "Launch Time": 345, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" | } - | } - | ]} + | ] + | }, + | "Task Metrics": { + | "Host Name": "localhost", + | "Executor Deserialize Time": 300, + | "Executor Run Time": 400, + | "Result Size": 500, + | "JVM GC Time": 600, + | "Result Serialization Time": 700, + | "Memory Bytes Spilled": 800, + | "Disk Bytes Spilled": 0, + | "Shuffle Write Metrics": { + | "Shuffle Bytes Written": 1200, + | "Shuffle Write Time": 1500 + | }, + | "Input Metrics": { + | "Data Read Method": "Hadoop", + | "Bytes Read": 2100 + | }, + | "Updated Blocks": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Tachyon Size": 0, + | "Disk Size": 0 + | } + | } + | ] + | } |} """ private val jobStartJsonString = """ - {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": - {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + |{ + | "Event": "SparkListenerJobStart", + | "Job ID": 10, + | "Stage IDs": [ + | 1, + | 2, + | 3, + | 4 + | ], + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} """ private val jobEndJsonString = """ - {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}} + |{ + | "Event": "SparkListenerJobEnd", + | "Job ID": 20, + | "Job Result": { + | "Result": "JobSucceeded" + | } + |} """ private val environmentUpdateJsonString = """ - {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s", - "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s, - regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"}, - "Classpath Entries":{"Super library":"/tmp/super_library"}} + |{ + | "Event": "SparkListenerEnvironmentUpdate", + | "JVM Information": { + | "GC speed": "9999 objects/s", + | "Java home": "Land of coffee" + | }, + | "Spark Properties": { + | "Job throughput": "80000 jobs/s, regardless of job type" + | }, + | "System Properties": { + | "Username": "guest", + | "Password": "guest" + | }, + | "Classpath Entries": { + | "Super library": "/tmp/super_library" + | } + |} """ private val blockManagerAddedJsonString = """ - {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars", - "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} + |{ + | "Event": "SparkListenerBlockManagerAdded", + | "Block Manager ID": { + | "Executor ID": "Stars", + | "Host": "In your multitude...", + | "Port": 300, + | "Netty Port": 400 + | }, + | "Maximum Memory": 500 + |} """ private val blockManagerRemovedJsonString = """ - {"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"Scarce", - "Host":"to be counted...","Port":100,"Netty Port":200}} + |{ + | "Event": "SparkListenerBlockManagerRemoved", + | "Block Manager ID": { + | "Executor ID": "Scarce", + | "Host": "to be counted...", + | "Port": 100, + | "Netty Port": 200 + | } + |} """ private val unpersistRDDJsonString = """ - {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} + |{ + | "Event": "SparkListenerUnpersistRDD", + | "RDD ID": 12345 + |} """ private val applicationStartJsonString = """ - {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, - "User":"Garfield"} + |{ + | "Event": "SparkListenerApplicationStart", + | "App Name": "The winner of all", + | "Timestamp": 42, + | "User": "Garfield" + |} """ private val applicationEndJsonString = """ - {"Event":"SparkListenerApplicationEnd","Timestamp":42} + |{ + | "Event": "SparkListenerApplicationEnd", + | "Timestamp": 42 + |} """ } From 5e57089a8eefd0939089a26e57b96f08e75968f6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 16:56:13 -0700 Subject: [PATCH 070/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit c204a742a9eb9d3fd318e0f059bd00cbfb8b2c14. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7f1172ec2092d..9d8a36d9d5b82 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b127136e3f5a0..6563f4d73da01 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 1479326af0ed9..359f102bf8f7d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-snapshot1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml From 2bcabcd6a732fa235d2b1279830809f394521fab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 16:56:18 -0700 Subject: [PATCH 071/192] Revert "[maven-release-plugin] prepare release v1.1.0-snapshot1" This reverts commit d428d88418d385d1d04e1b0adcb6b068efe9c7b0. --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 +++++---- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 +++-- sql/core/pom.xml | 5 +++-- sql/hive-thriftserver/pom.xml | 5 +++-- sql/hive/pom.xml | 5 +++-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 38 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9d8a36d9d5b82..0c68defa5e101 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6563f4d73da01..c532705f3950c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..a54b34235dfb4 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 359f102bf8f7d..b8df3d025cfbf 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-snapshot1 + HEAD @@ -879,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From f8bcb12c1820402824a8d65dcbb60189e08679c6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 Aug 2014 17:07:39 -0700 Subject: [PATCH 072/192] [SPARK-3140] Clarify confusing PySpark exception message We read the py4j port from the stdout of the `bin/spark-submit` subprocess. If there is interference in stdout (e.g. a random echo in `spark-submit`), we throw an exception with a warning message. We do not, however, distinguish between this case from the case where no stdout is produced at all. I wasted a non-trivial amount of time being baffled by this exception in search of places where I print random whitespace (in vain, of course). A clearer exception message that distinguishes between these cases will prevent similar headaches that I have gone through. Author: Andrew Or Closes #2067 from andrewor14/python-exception and squashes the following commits: 742f823 [Andrew Or] Further clarify warning messages e96a7a0 [Andrew Or] Distinguish between unexpected output and no output at all (cherry picked from commit ba3c730e35bcdb662396955c3cc6f7de628034c8) Signed-off-by: Andrew Or --- python/pyspark/java_gateway.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c7f7c1fe591b0..6f4f62f23bc4d 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -54,12 +54,19 @@ def preexec_func(): gateway_port = proc.stdout.readline() gateway_port = int(gateway_port) except ValueError: + # Grab the remaining lines of stdout (stdout, _) = proc.communicate() exit_code = proc.poll() error_msg = "Launching GatewayServer failed" - error_msg += " with exit code %d! " % exit_code if exit_code else "! " - error_msg += "(Warning: unexpected output detected.)\n\n" - error_msg += gateway_port + stdout + error_msg += " with exit code %d!\n" % exit_code if exit_code else "!\n" + error_msg += "Warning: Expected GatewayServer to output a port, but found " + if gateway_port == "" and stdout == "": + error_msg += "no output.\n" + else: + error_msg += "the following:\n\n" + error_msg += "--------------------------------------------------------------\n" + error_msg += gateway_port + stdout + error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) # Create a thread to echo output from the GatewayServer, which is required From 1af68caf68d6d34f588723184dc2f75d7578b1d9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 Aug 2014 17:41:36 -0700 Subject: [PATCH 073/192] [SPARK-3143][MLLIB] add tf-idf user guide Moved TF-IDF before Word2Vec because the former is more basic. I also added a link for Word2Vec. atalwalkar Author: Xiangrui Meng Closes #2061 from mengxr/tfidf-doc and squashes the following commits: ca04c70 [Xiangrui Meng] address comments a5ea4b4 [Xiangrui Meng] add tf-idf user guide (cherry picked from commit e1571874f26c1df2dfd5ac2959612372716cd2d8) Signed-off-by: Xiangrui Meng --- docs/mllib-feature-extraction.md | 83 ++++++++++++++++++++++++++++++-- 1 file changed, 80 insertions(+), 3 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 4b3cb715c58c7..2031b96235ee9 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -7,9 +7,88 @@ displayTitle: MLlib - Feature Extraction * Table of contents {:toc} + +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature +vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. +Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, +while document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. +If we only use term frequency to measure the importance, it is very easy to over-emphasize terms that +appear very often but carry little information about the document, e.g., "a", "the", and "of". +If a term appears very often across the corpus, it means it doesn't carry special information about +a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. +Since logarithm is used, if a term appears in all documents, its IDF value becomes 0. +Note that a smoothing term is applied to avoid dividing by zero for terms outside the corpus. +The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. + +Our implementation of term frequency utilizes the +[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. +Then term frequencies are calculated based on the mapped indices. +This approach avoids the need to compute a global term-to-index map, +which can be expensive for a large corpus, but it suffers from potential hash collisions, +where different raw features may become the same term after hashing. +To reduce the chance of collision, we can increase the target feature dimension, i.e., +the number of buckets of the hash table. +The default feature dimension is `$2^{20} = 1,048,576$`. + +**Note:** MLlib doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). + +
    +
    + +TF and IDF are implemented in [HashingTF](api/scala/index.html#org.apache.spark.mllib.feature.HashingTF) +and [IDF](api/scala/index.html#org.apache.spark.mllib.feature.IDF). +`HashingTF` takes an `RDD[Iterable[_]]` as the input. +Each record could be an iterable of strings or other types. + +{% highlight scala %} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.mllib.feature.HashingTF +import org.apache.spark.mllib.linalg.Vector + +val sc: SparkContext = ... + +// Load documents (one per line). +val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) + +val hashingTF = new HashingTF() +val tf: RDD[Vector] = hasingTF.transform(documents) +{% endhighlight %} + +While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: +first to compute the IDF vector and second to scale the term frequencies by IDF. + +{% highlight scala %} +import org.apache.spark.mllib.feature.IDF + +// ... continue from the previous example +tf.cache() +val idf = new IDF().fit(tf) +val tfidf: RDD[Vector] = idf.transform(tf) +{% endhighlight %} +
    +
    + ## Word2Vec -Word2Vec computes distributed vector representation of words. The main advantage of the distributed +[Word2Vec](https://code.google.com/p/word2vec/) computes distributed vector representation of words. +The main advantage of the distributed representations is that similar words are close in the vector space, which makes generalization to novel patterns easier and model estimation more robust. Distributed vector representation is showed to be useful in many natural language processing applications such as named entity @@ -69,5 +148,3 @@ for((synonym, cosineSimilarity) <- synonyms) { {% endhighlight %} - -## TFIDF \ No newline at end of file From eba399b3c6768f5106cbc17752630fa81d9cdce4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 Aug 2014 17:47:39 -0700 Subject: [PATCH 074/192] [SPARK-2843][MLLIB] add a section about regularization parameter in ALS atalwalkar srowen Author: Xiangrui Meng Closes #2064 from mengxr/als-doc and squashes the following commits: b2e20ab [Xiangrui Meng] introduced -> discussed 98abdd7 [Xiangrui Meng] add reference 339bd08 [Xiangrui Meng] add a section about regularization parameter in ALS (cherry picked from commit e0f946265b9ea5bc48849cf7794c2c03d5e29fba) Signed-off-by: Xiangrui Meng --- docs/mllib-collaborative-filtering.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index ab10b2f01f87b..d5c539db791be 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -43,6 +43,17 @@ level of confidence in observed user preferences, rather than explicit ratings g model then tries to find latent factors that can be used to predict the expected preference of a user for an item. +### Scaling of the regularization parameter + +Since v1.1, we scale the regularization parameter `lambda` in solving each least squares problem by +the number of ratings the user generated in updating user factors, +or the number of ratings the product received in updating product factors. +This approach is named "ALS-WR" and discussed in the paper +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +It makes `lambda` less dependent on the scale of the dataset. +So we can apply the best parameter learned from a sampled subset to the full dataset +and expect similar performance. + ## Examples
    From 3f91e9dc2563f3c5c473c781bd3078cc620ff880 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Aug 2014 16:34:53 -0700 Subject: [PATCH 075/192] [HOTFIX][Streaming] Handle port collisions in flume polling test This is failing my tests in #1777. @tdas Author: Andrew Or Closes #1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits: ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions 54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test 664095c [Andrew Or] Tone down bind exception message af3ddc9 [Andrew Or] Handle port collisions in flume polling test --- .../flume/FlumePollingStreamSuite.scala | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 2e4ac7cfbf263..e3a5bdcd24868 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} import org.apache.spark.streaming.flume.sink._ +import org.apache.spark.util.Utils class FlumePollingStreamSuite extends TestSuiteBase { @@ -45,8 +46,37 @@ class FlumePollingStreamSuite extends TestSuiteBase { val eventsPerBatch = 100 val totalEventsPerChannel = batchCount * eventsPerBatch val channelCapacity = 5000 + val maxAttempts = 5 test("flume polling test") { + testMultipleTimes(testFlumePolling) + } + + test("flume polling test multiple hosts") { + testMultipleTimes(testFlumePollingMultipleHost) + } + + /** + * Run the given test until no more java.net.BindException's are thrown. + * Do this only up to a certain attempt limit. + */ + private def testMultipleTimes(test: () => Unit): Unit = { + var testPassed = false + var attempt = 0 + while (!testPassed && attempt < maxAttempts) { + try { + test() + testPassed = true + } catch { + case e: Exception if Utils.isBindCollision(e) => + logWarning("Exception when running flume polling test: " + e) + attempt += 1 + } + } + assert(testPassed, s"Test failed after $attempt attempts!") + } + + private def testFlumePolling(): Unit = { val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) @@ -80,7 +110,7 @@ class FlumePollingStreamSuite extends TestSuiteBase { channel.stop() } - test("flume polling test multiple hosts") { + private def testFlumePollingMultipleHost(): Unit = { val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) From 44856654c81ceb92ef6380691027744d4bf76589 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 17 Aug 2014 19:50:31 -0700 Subject: [PATCH 076/192] [HOTFIX][STREAMING] Allow the JVM/Netty to decide which port to bind to in Flume Polling Tests. Author: Hari Shreedharan Closes #1820 from harishreedharan/use-free-ports and squashes the following commits: b939067 [Hari Shreedharan] Remove unused import. 67856a8 [Hari Shreedharan] Remove findFreePort. 0ea51d1 [Hari Shreedharan] Make some changes to getPort to use map on the serverOpt. 1fb0283 [Hari Shreedharan] Merge branch 'master' of https://github.com/apache/spark into use-free-ports b351651 [Hari Shreedharan] Allow Netty to choose port, and query it to decide the port to bind to. Leaving findFreePort as is, if other tests want to use it at some point. e6c9620 [Hari Shreedharan] Making sure the second sink uses the correct port. 11c340d [Hari Shreedharan] Add info about race condition to scaladoc. e89d135 [Hari Shreedharan] Adding Scaladoc. 6013bb0 [Hari Shreedharan] [STREAMING] Find free ports to use before attempting to create Flume Sink in Flume Polling Suite --- .../streaming/flume/sink/SparkSink.scala | 8 +++ .../flume/FlumePollingStreamSuite.scala | 55 +++++++++---------- 2 files changed, 34 insertions(+), 29 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 1a61b36910a95..98ae7d783aec8 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -130,6 +130,14 @@ class SparkSink extends AbstractSink with Logging with Configurable { blockingLatch.await() Status.BACKOFF } + + private[flume] def getPort(): Int = { + serverOpt + .map(_.getPort) + .getOrElse( + throw new RuntimeException("Server was not started!") + ) + } } /** diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index e3a5bdcd24868..32a19787a28e1 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -22,6 +22,8 @@ import java.net.InetSocketAddress import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} import java.util.Random +import org.apache.spark.TestUtils + import scala.collection.JavaConversions._ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -39,9 +41,6 @@ import org.apache.spark.util.Utils class FlumePollingStreamSuite extends TestSuiteBase { - val random = new Random() - /** Return a port in the ephemeral range. */ - def getTestPort = random.nextInt(16382) + 49152 val batchCount = 5 val eventsPerBatch = 100 val totalEventsPerChannel = batchCount * eventsPerBatch @@ -77,17 +76,6 @@ class FlumePollingStreamSuite extends TestSuiteBase { } private def testFlumePolling(): Unit = { - val testPort = getTestPort - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), - StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) - outputStream.register() - // Start the channel and sink. val context = new Context() context.put("capacity", channelCapacity.toString) @@ -98,10 +86,19 @@ class FlumePollingStreamSuite extends TestSuiteBase { val sink = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink, context) sink.setChannel(channel) sink.start() + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", sink.getPort())), + StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() ssc.start() writeAndVerify(Seq(channel), ssc, outputBuffer) @@ -111,18 +108,6 @@ class FlumePollingStreamSuite extends TestSuiteBase { } private def testFlumePollingMultipleHost(): Unit = { - val testPort = getTestPort - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, - eventsPerBatch, 5) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) - outputStream.register() - // Start the channel and sink. val context = new Context() context.put("capacity", channelCapacity.toString) @@ -136,17 +121,29 @@ class FlumePollingStreamSuite extends TestSuiteBase { val sink = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink, context) sink.setChannel(channel) sink.start() val sink2 = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort + 1)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink2, context) sink2.setChannel(channel2) sink2.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val addresses = Seq(sink.getPort(), sink2.getPort()).map(new InetSocketAddress("localhost", _)) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + eventsPerBatch, 5) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + ssc.start() writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) assertChannelIsEmpty(channel) From 1d5e84a99076d3e0168dd2f4626c7911e7ba49e7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 22:24:22 -0700 Subject: [PATCH 077/192] HOTFIX:Temporarily removing flume sink test in 1.1 branch --- .../streaming/flume/sink/SparkSinkSuite.scala | 204 ------------------ 1 file changed, 204 deletions(-) delete mode 100644 external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala deleted file mode 100644 index 44b27edf85ce8..0000000000000 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.streaming.flume.sink - -import java.net.InetSocketAddress -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} - -import scala.collection.JavaConversions._ -import scala.concurrent.{ExecutionContext, Future} -import scala.util.{Failure, Success} - -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.flume.Context -import org.apache.flume.channel.MemoryChannel -import org.apache.flume.event.EventBuilder -import org.apache.spark.streaming.TestSuiteBase -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory - -class SparkSinkSuite extends TestSuiteBase { - val eventsPerBatch = 1000 - val channelCapacity = 5000 - - test("Success") { - val (channel, sink) = initializeChannelAndSink() - channel.start() - sink.start() - - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - client.ack(events.getSequenceNumber) - assert(events.getEvents.size() === 1000) - assertChannelIsEmpty(channel) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Nack") { - val (channel, sink) = initializeChannelAndSink() - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - client.nack(events.getSequenceNumber) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Timeout") { - val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig - .CONF_TRANSACTION_TIMEOUT -> 1.toString)) - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - Thread.sleep(1000) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Multiple consumers") { - testMultipleConsumers(failSome = false) - } - - test("Multiple consumers with some failures") { - testMultipleConsumers(failSome = true) - } - - def testMultipleConsumers(failSome: Boolean): Unit = { - implicit val executorContext = ExecutionContext - .fromExecutorService(Executors.newFixedThreadPool(5)) - val (channel, sink) = initializeChannelAndSink() - channel.start() - sink.start() - (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - val transceiversAndClients = getTransceiverAndClient(address, 5) - val batchCounter = new CountDownLatch(5) - val counter = new AtomicInteger(0) - transceiversAndClients.foreach(x => { - Future { - val client = x._2 - val events = client.getEventBatch(1000) - if (!failSome || counter.getAndIncrement() % 2 == 0) { - client.ack(events.getSequenceNumber) - } else { - client.nack(events.getSequenceNumber) - throw new RuntimeException("Sending NACK for failure!") - } - events - }.onComplete { - case Success(events) => - assert(events.getEvents.size() === 1000) - batchCounter.countDown() - case Failure(t) => - // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout - batchCounter.countDown() - } - }) - batchCounter.await() - TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. - executorContext.shutdown() - if(failSome) { - assert(availableChannelSlots(channel) === 3000) - } else { - assertChannelIsEmpty(channel) - } - sink.stop() - channel.stop() - transceiversAndClients.foreach(x => x._1.close()) - } - - private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, - SparkSink) = { - val channel = new MemoryChannel() - val channelContext = new Context() - - channelContext.put("capacity", channelCapacity.toString) - channelContext.put("transactionCapacity", 1000.toString) - channelContext.put("keep-alive", 0.toString) - channelContext.putAll(overrides) - channel.configure(channelContext) - - val sink = new SparkSink() - val sinkContext = new Context() - sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") - sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) - sink.configure(sinkContext) - sink.setChannel(channel) - (channel, sink) - } - - private def putEvents(ch: MemoryChannel, count: Int): Unit = { - val tx = ch.getTransaction - tx.begin() - (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) - tx.commit() - tx.close() - } - - private def getTransceiverAndClient(address: InetSocketAddress, - count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { - - (1 to count).map(_ => { - lazy val channelFactoryExecutor = - Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). - setNameFormat("Flume Receiver Channel Thread - %d").build()) - lazy val channelFactory = - new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) - val transceiver = new NettyTransceiver(address, channelFactory) - val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - (transceiver, client) - }) - } - - private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { - assert(availableChannelSlots(channel) === channelCapacity) - } - - private def availableChannelSlots(channel: MemoryChannel): Int = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") - queueRemaining.setAccessible(true) - val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] - } -} From e1535ad3c6f7400f2b7915ea91da9c60510557ba Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 21 Aug 2014 05:54:41 +0000 Subject: [PATCH 078/192] [maven-release-plugin] prepare release v1.1.0-snapshot2 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 0c68defa5e101..9d8a36d9d5b82 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index c532705f3950c..6563f4d73da01 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index b8df3d025cfbf..be190d6548e80 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-snapshot2 @@ -880,7 +879,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From 9af3fb7385d1f9f221962f1d2d725ff79bd82033 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 21 Aug 2014 05:54:48 +0000 Subject: [PATCH 079/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9d8a36d9d5b82..7f1172ec2092d 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6563f4d73da01..b127136e3f5a0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index be190d6548e80..1479326af0ed9 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-snapshot2 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From da0a701204ae057581ed2d41eba5bb610e36c864 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 12:18:41 -0700 Subject: [PATCH 080/192] BUILD: Bump Hadoop versions in the release build. Also, minor modifications to the MapR profile. --- dev/create-release/create-release.sh | 10 +++---- pom.xml | 39 +++++++++++++++++++++++++--- 2 files changed, 40 insertions(+), 9 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 905dec0ced383..eab6313733dfd 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,11 +118,11 @@ make_binary_release() { } make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2" \ - "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & -make_binary_release "hadoop2-without-hive" \ - "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Pyarn" & +make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Pyarn" & +make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & +make_binary_release "mapr3" "-Pmapr3 -Pyarn -Phive" & +make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive" & wait # Copy data diff --git a/pom.xml b/pom.xml index 1479326af0ed9..bc3aa060e9dfc 100644 --- a/pom.xml +++ b/pom.xml @@ -1114,18 +1114,49 @@ - mapr + mapr3 false 1.0.3-mapr-3.0.3 - 2.3.0-mapr-4.0.0-beta - 0.94.17-mapr-1403 - 3.4.5-mapr-1401 + 2.3.0-mapr-4.0.0-FCS + 0.94.17-mapr-1405 + 3.4.5-mapr-1406 + + mapr4 + + false + + + 2.3.0-mapr-4.0.0-FCS + 2.3.0-mapr-4.0.0-FCS + 0.94.17-mapr-1405-4.0.0-FCS + 3.4.5-mapr-1406 + + + + org.apache.curator + curator-recipes + 2.4.0 + + + org.apache.zookeeper + zookeeper + + + + + org.apache.zookeeper + zookeeper + 3.4.5-mapr-1406 + + + + hadoop-provided From 1e5d9cbb499199304aa8820114fa77dc7a3f0224 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 21 Aug 2014 00:17:29 -0700 Subject: [PATCH 081/192] [SPARK-2840] [mllib] DecisionTree doc update (Java, Python examples) Updated DecisionTree documentation, with examples for Java, Python. Added same Java example to code as well. CC: @mengxr @manishamde @atalwalkar Author: Joseph K. Bradley Closes #2063 from jkbradley/dt-docs and squashes the following commits: 2dd2c19 [Joseph K. Bradley] Last updates based on github review. 9dd1b6b [Joseph K. Bradley] Updated decision tree doc. d802369 [Joseph K. Bradley] Updates based on comments: cache data, corrected doc text. b9bee04 [Joseph K. Bradley] Updated DT examples 57eee9f [Joseph K. Bradley] Created JavaDecisionTree example from example in docs, and corrected doc example as needed. d939a92 [Joseph K. Bradley] Updated DecisionTree documentation. Added Java, Python examples. (cherry picked from commit 050f8d01e47b9b67b02ce50d83fb7b4e528b7204) Signed-off-by: Xiangrui Meng --- docs/mllib-decision-tree.md | 352 ++++++++++++++---- .../examples/mllib/JavaDecisionTree.java | 116 ++++++ 2 files changed, 399 insertions(+), 69 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index c01a92a9a1b26..1166d9cd150c4 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -7,20 +7,26 @@ displayTitle: MLlib - Decision Tree * Table of contents {:toc} -Decision trees and their ensembles are popular methods for the machine learning tasks of +[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) +and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, -handle categorical variables, extend to the multiclass classification setting, do not require +handle categorical features, extend to the multiclass classification setting, do not require feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble -algorithms such as decision forest and boosting are among the top performers for classification and +algorithms such as random forests and boosting are among the top performers for classification and regression tasks. +MLlib supports decision trees for binary and multiclass classification and for regression, +using both continuous and categorical features. The implementation partitions data by rows, +allowing distributed training with millions of instances. + ## Basic algorithm The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature -space by choosing a single element from the *best split set* where each element of the set maximizes -the information gain at a tree node. In other words, the split chosen at each tree node is chosen -from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information -gain when a split `$s$` is applied to a dataset `$D$`. +space. The tree predicts the same label for each bottommost (leaf) partition. +Each partition is chosen greedily by selecting the *best split* from a set of possible splits, +in order to maximize the information gain at a tree node. In other words, the split chosen at each +tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` +is the information gain when a split `$s$` is applied to a dataset `$D$`. ### Node impurity and information gain @@ -52,9 +58,10 @@ impurity measure for regression (variance). -The *information gain* is the difference in the parent node impurity and the weighted sum of the two -child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two -datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: +The *information gain* is the difference between the parent node impurity and the weighted sum of +the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` +into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, +respectively, the information gain is: `$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` @@ -62,14 +69,15 @@ datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, **Continuous features** -For small datasets in single machine implementations, the split candidates for each continuous +For small datasets in single-machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. -Finding ordered unique feature values is computationally intensive for large distributed -datasets. One can get an approximate set of split candidates by performing a quantile calculation -over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such -bins can be specified using the `maxBins` parameters. +Sorting feature values is expensive for large distributed datasets. +This implementation computes an approximate set of split candidates by performing a quantile +calculation over a sampled fraction of the data. +The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of @@ -77,109 +85,315 @@ bins if the condition is not satisfied. **Categorical features** -For `$M$` categorical feature values, one could come up with `$2^(M-1)-1$` split candidates. For -binary classification, we can reduce the number of split candidates to `$M-1$` by ordering the -categorical feature values by the proportion of labels falling in one of the two classes (see -Section 9.2.4 in +For a categorical feature with `$M$` possible values (categories), one could come up with +`$2^{M-1}-1$` split candidates. For binary (0/1) classification and regression, +we can reduce the number of split candidates to `$M-1$` by ordering the +categorical feature values by the average label. (See Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for -details). For example, for a binary classification problem with one categorical feature with three -categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are ordered as A followed by C followed B or A, C, B. The two split candidates are A \| C, B -and A , C \| B where \| denotes the split. A similar heuristic is used for multiclass classification -when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value -is used for ordering. +details.) For example, for a binary classification problem with one categorical feature with three +categories A, B and C whose corresponding proportions of label 1 are 0.2, 0.6 and 0.4, the categorical +features are ordered as A, C, B. The two split candidates are A \| C, B +and A , C \| B where \| denotes the split. + +In multiclass classification, all `$2^{M-1}-1$` possible splits are used whenever possible. +When `$2^{M-1}-1$` is greater than the `maxBins` parameter, we use a (heuristic) method +similar to the method used for binary classification and regression. +The `$M$` categorical feature values are ordered by impurity, +and the resulting `$M-1$` split candidates are considered. ### Stopping rule The recursive tree construction is stopped at a node when one of the two conditions is met: -1. The node depth is equal to the `maxDepth` training parameter +1. The node depth is equal to the `maxDepth` training parameter. 2. No split candidate leads to an information gain at the node. +## Implementation details + ### Max memory requirements -For faster processing, the decision tree algorithm performs simultaneous histogram computations for all nodes at each level of the tree. This could lead to high memory requirements at deeper levels of the tree leading to memory overflow errors. To alleviate this problem, a 'maxMemoryInMB' training parameter is provided which specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation crosses the `maxMemoryInMB` threshold, the node training tasks at each subsequent level is split into smaller tasks. +For faster processing, the decision tree algorithm performs simultaneous histogram computations for +all nodes at each level of the tree. This could lead to high memory requirements at deeper levels +of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` +training parameter specifies the maximum amount of memory at the workers (twice as much at the +master) to be allocated to the histogram computation. The default value is conservatively chosen to +be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each +subsequent level are split into smaller tasks. + +Note that, if you have a large amount of memory, increasing `maxMemoryInMB` can lead to faster +training by requiring fewer passes over the data. + +### Binning feature values + +Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained +split decisions. However, it also increases computation and communication. + +Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for +any categorical feature. + +### Scaling -### Practical limitations +Computation scales approximately linearly in the number of training instances, +in the number of features, and in the `maxBins` parameter. +Communication scales approximately linearly in the number of features and in `maxBins`. -1. The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. -2. Python is not supported in this release. +The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. ## Examples ### Classification -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform classification using a decision tree using Gini impurity as an impurity measure and a +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree with Gini impurity as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy.
    +
    {% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "gini" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelAndPreds = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / data.count println("Training Error = " + trainErr) +println("Learned classification tree model:\n" + model) +{% endhighlight %} +
    + +
    +{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +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.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Integer numClasses = 2; +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "gini"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model for classification. +final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); +System.out.println("Training error: " + trainErr); +System.out.println("Learned classification tree model:\n" + model); +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, + impurity='gini', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(data.count()) +print('Training Error = ' + str(trainErr)) +print('Learned classification tree model:') +print(model) {% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala.
    +
    ### Regression -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform regression using a decision tree using variance as an impurity measure and a maximum tree +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree with variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
    +
    {% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "variance" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelsAndPredictions = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("training Mean Squared Error = " + MSE) +val trainMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Training Mean Squared Error = " + trainMSE) +println("Learned regression tree model:\n" + model) {% endhighlight %}
    + +
    +{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +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.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "variance"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model. +final DecisionTreeModel model = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainMSE = + predictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); +System.out.println("Training Mean Squared Error: " + trainMSE); +System.out.println("Learned regression tree model:\n" + model); +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, + impurity='variance', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(data.count()) +print('Training Mean Squared Error = ' + str(trainMSE)) +print('Learned regression tree model:') +print(model) +{% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala. +
    +
    diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java new file mode 100644 index 0000000000000..e4468e8bf1744 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -0,0 +1,116 @@ +/* + * 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 java.util.HashMap; + +import scala.Tuple2; + +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +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.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +/** + * Classification and regression using decision trees. + */ +public final class JavaDecisionTree { + + public static void main(String[] args) { + String datapath = "data/mllib/sample_libsvm_data.txt"; + if (args.length == 1) { + datapath = args[0]; + } else if (args.length > 1) { + System.err.println("Usage: JavaDecisionTree "); + System.exit(1); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + + // Compute the number of classes from the data. + Integer numClasses = data.map(new Function() { + @Override public Double call(LabeledPoint p) { + return p.label(); + } + }).countByValue().size(); + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + HashMap categoricalFeaturesInfo = new HashMap(); + String impurity = "gini"; + Integer maxDepth = 5; + Integer maxBins = 100; + + // Train a DecisionTree model for classification. + final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); + System.out.println("Training error: " + trainErr); + System.out.println("Learned classification tree model:\n" + model); + + // Train a DecisionTree model for regression. + impurity = "variance"; + final DecisionTreeModel regressionModel = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD regressorPredictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(regressionModel.predict(p.features()), p.label()); + } + }); + Double trainMSE = + regressorPredictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); + System.out.println("Training Mean Squared Error: " + trainMSE); + System.out.println("Learned regression tree model:\n" + regressionModel); + + sc.stop(); + } +} From 385c4f2af5996844b9761942643f71a6544e1dd8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 22 Aug 2014 21:31:52 -0700 Subject: [PATCH 082/192] Revert "HOTFIX:Temporarily removing flume sink test in 1.1 branch" This reverts commit 1d5e84a99076d3e0168dd2f4626c7911e7ba49e7. --- .../streaming/flume/sink/SparkSinkSuite.scala | 204 ++++++++++++++++++ 1 file changed, 204 insertions(+) create mode 100644 external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala new file mode 100644 index 0000000000000..44b27edf85ce8 --- /dev/null +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -0,0 +1,204 @@ +/* + * 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.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} + +import scala.collection.JavaConversions._ +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.event.EventBuilder +import org.apache.spark.streaming.TestSuiteBase +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +class SparkSinkSuite extends TestSuiteBase { + val eventsPerBatch = 1000 + val channelCapacity = 5000 + + test("Success") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + client.ack(events.getSequenceNumber) + assert(events.getEvents.size() === 1000) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Nack") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + client.nack(events.getSequenceNumber) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Timeout") { + val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig + .CONF_TRANSACTION_TIMEOUT -> 1.toString)) + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + Thread.sleep(1000) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Multiple consumers") { + testMultipleConsumers(failSome = false) + } + + test("Multiple consumers with some failures") { + testMultipleConsumers(failSome = true) + } + + def testMultipleConsumers(failSome: Boolean): Unit = { + implicit val executorContext = ExecutionContext + .fromExecutorService(Executors.newFixedThreadPool(5)) + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + val transceiversAndClients = getTransceiverAndClient(address, 5) + val batchCounter = new CountDownLatch(5) + val counter = new AtomicInteger(0) + transceiversAndClients.foreach(x => { + Future { + val client = x._2 + val events = client.getEventBatch(1000) + if (!failSome || counter.getAndIncrement() % 2 == 0) { + client.ack(events.getSequenceNumber) + } else { + client.nack(events.getSequenceNumber) + throw new RuntimeException("Sending NACK for failure!") + } + events + }.onComplete { + case Success(events) => + assert(events.getEvents.size() === 1000) + batchCounter.countDown() + case Failure(t) => + // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout + batchCounter.countDown() + } + }) + batchCounter.await() + TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. + executorContext.shutdown() + if(failSome) { + assert(availableChannelSlots(channel) === 3000) + } else { + assertChannelIsEmpty(channel) + } + sink.stop() + channel.stop() + transceiversAndClients.foreach(x => x._1.close()) + } + + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, + SparkSink) = { + val channel = new MemoryChannel() + val channelContext = new Context() + + channelContext.put("capacity", channelCapacity.toString) + channelContext.put("transactionCapacity", 1000.toString) + channelContext.put("keep-alive", 0.toString) + channelContext.putAll(overrides) + channel.configure(channelContext) + + val sink = new SparkSink() + val sinkContext = new Context() + sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") + sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) + sink.configure(sinkContext) + sink.setChannel(channel) + (channel, sink) + } + + private def putEvents(ch: MemoryChannel, count: Int): Unit = { + val tx = ch.getTransaction + tx.begin() + (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) + tx.commit() + tx.close() + } + + private def getTransceiverAndClient(address: InetSocketAddress, + count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { + + (1 to count).map(_ => { + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + val transceiver = new NettyTransceiver(address, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + (transceiver, client) + }) + } + + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { + assert(availableChannelSlots(channel) === channelCapacity) + } + + private def availableChannelSlots(channel: MemoryChannel): Int = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] + } +} From cd73631b15f080405e04203bf15fbd31c65eb64a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 22 Aug 2014 21:34:48 -0700 Subject: [PATCH 083/192] [SPARK-3169] Removed dependency on spark streaming test from spark flume sink Due to maven bug https://jira.codehaus.org/browse/MNG-1378, maven could not resolve spark streaming classes required by the spark-streaming test-jar dependency of external/flume-sink. There is no particular reason that the external/flume-sink has to depend on Spark Streaming at all, so I am eliminating this dependency. Also I have removed the exclusions present in the Flume dependencies, as there is no reason to exclude them (they were excluded in the external/flume module to prevent dependency collisions with Spark). Since Jenkins will test the sbt build and the unit test, I only tested maven compilation locally. Author: Tathagata Das Closes #2101 from tdas/spark-sink-pom-fix and squashes the following commits: 8f42621 [Tathagata Das] Added Flume sink exclusions back, and added netty to test dependencies 93b559f [Tathagata Das] Removed dependency on spark streaming test from spark flume sink (cherry picked from commit 3004074152b7261c2a968bb8e94ec7c41a7b43c1) Signed-off-by: Patrick Wendell --- external/flume-sink/pom.xml | 18 ++++++++++++------ .../streaming/flume/sink/SparkSinkSuite.scala | 10 +++++----- 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7f1172ec2092d..4373cb058e7c0 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -62,7 +62,7 @@ org.apache.thrift libthrift - + org.scala-lang @@ -71,13 +71,19 @@ org.scalatest scalatest_${scala.binary.version} + test - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test + + io.netty + netty + 3.4.0.Final + test diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 44b27edf85ce8..75a6668c6210b 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -30,14 +30,14 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.flume.Context import org.apache.flume.channel.MemoryChannel import org.apache.flume.event.EventBuilder -import org.apache.spark.streaming.TestSuiteBase import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.scalatest.FunSuite -class SparkSinkSuite extends TestSuiteBase { +class SparkSinkSuite extends FunSuite { val eventsPerBatch = 1000 val channelCapacity = 5000 - test("Success") { + test("Success with ack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -57,7 +57,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Nack") { + test("Failure with nack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -76,7 +76,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Timeout") { + test("Failure with timeout") { val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig .CONF_TRANSACTION_TIMEOUT -> 1.toString)) channel.start() From 568966018bff437f1d73cd59eb4681b2d3e87b48 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 22 Aug 2014 22:28:05 -0700 Subject: [PATCH 084/192] [SPARK-2963] REGRESSION - The description about how to build for using CLI and Thrift JDBC server is absent in proper document - The most important things I mentioned in #1885 is as follows. * People who build Spark is not always programmer. * If a person who build Spark is not a programmer, he/she won't read programmer's guide before building. So, how to build for using CLI and JDBC server is not only in programmer's guide. Author: Kousuke Saruta Closes #2080 from sarutak/SPARK-2963 and squashes the following commits: ee07c76 [Kousuke Saruta] Modified regression of the description about building for using Thrift JDBC server and CLI ed53329 [Kousuke Saruta] Modified description and notaton of proper noun 07c59fc [Kousuke Saruta] Added a description about how to build to use HiveServer and CLI for SparkSQL to building-with-maven.md 6e6645a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2963 c88fa93 [Kousuke Saruta] Added a description about building to use HiveServer and CLI for SparkSQL --- README.md | 5 ++++- docs/building-with-maven.md | 11 +++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 8906e4c1416b1..bec68e82fbeb5 100644 --- a/README.md +++ b/README.md @@ -118,7 +118,10 @@ If your project is built with Maven, add this to your POM file's ` ## A Note About Thrift JDBC server and CLI for Spark SQL Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about using the JDBC server. +See sql-programming-guide.md for more information about using the JDBC server and CLI. +You can use those features by setting `-Phive` when building Spark as follows. + + $ sbt/sbt -Phive assembly ## Configuration diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a7d7bd3ccb1f2..bce7412c7d4c9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,10 +96,13 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} -# Building Thrift JDBC server and CLI for Spark SQL - -Spark SQL supports Thrift JDBC server and CLI. See sql-programming-guide.md for -more information about the JDBC server. +# Building With Hive and JDBC Support +To enable Hive integration for Spark SQL along with its JDBC server and CLI, +add the `-Phive` profile to your existing build options. +{% highlight bash %} +# Apache Hadoop 2.4.X with Hive support +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package +{% endhighlight %} # Spark Tests in Maven From 9309786416c83b2f3401724fdeb19c2be07c0431 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 23 Aug 2014 12:46:41 -0700 Subject: [PATCH 085/192] [SQL] Make functionRegistry in HiveContext transient. Seems we missed `transient` for the `functionRegistry` in `HiveContext`. cc: marmbrus Author: Yin Huai Closes #2074 from yhuai/makeFunctionRegistryTransient and squashes the following commits: 6534e7d [Yin Huai] Make functionRegistry transient. (cherry picked from commit 2fb1c72ea21e137c8b60a72e5aecd554c71b16e1) Signed-off-by: Michael Armbrust --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ff32c7c90a0d2..29baefe714c64 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,6 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } // Note that HiveUDFs will be overridden by functions registered in this context. + @transient override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry From 7112da8fe8d382a1180118f206db78f8e610d83f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 23 Aug 2014 16:19:10 -0700 Subject: [PATCH 086/192] [SPARK-2554][SQL] CountDistinct partial aggregation and object allocation improvements Author: Michael Armbrust Author: Gregory Owen Closes #1935 from marmbrus/countDistinctPartial and squashes the following commits: 5c7848d [Michael Armbrust] turn off caching in the constructor 8074a80 [Michael Armbrust] fix tests 32d216f [Michael Armbrust] reynolds comments c122cca [Michael Armbrust] Address comments, add tests b2e8ef3 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial fae38f4 [Michael Armbrust] Fix style fdca896 [Michael Armbrust] cleanup 93d0f64 [Michael Armbrust] metastore concurrency fix. db44a30 [Michael Armbrust] JIT hax. 3868f6c [Michael Armbrust] Merge pull request #9 from GregOwen/countDistinctPartial c9e67de [Gregory Owen] Made SpecificRow and types serializable by Kryo 2b46c4b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial 8ff6402 [Michael Armbrust] Add specific row. 58d15f1 [Michael Armbrust] disable codegen logging 87d101d [Michael Armbrust] Fix isNullAt bug abee26d [Michael Armbrust] WIP 27984d0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial 57ae3b1 [Michael Armbrust] Fix order dependent test b3d0f64 [Michael Armbrust] Add golden files. c1f7114 [Michael Armbrust] Improve tests / fix serialization. f31b8ad [Michael Armbrust] more fixes 38c7449 [Michael Armbrust] comments and style 9153652 [Michael Armbrust] better toString d494598 [Michael Armbrust] Fix tests now that the planner is better 41fbd1d [Michael Armbrust] Never try and create an empty hash set. 050bb97 [Michael Armbrust] Skip no-arg constructors for kryo, bd08239 [Michael Armbrust] WIP 213ada8 [Michael Armbrust] First draft of partially aggregated and code generated count distinct / max (cherry picked from commit 7e191fe29bb09a8560cd75d453c4f7f662dff406) Signed-off-by: Michael Armbrust --- .../sql/catalyst/expressions/Projection.scala | 344 +++++++++++++++++- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../catalyst/expressions/SpecificRow.scala | 307 ++++++++++++++++ .../sql/catalyst/expressions/aggregates.scala | 93 ++++- .../sql/catalyst/expressions/arithmetic.scala | 31 ++ .../expressions/codegen/CodeGenerator.scala | 93 ++++- .../codegen/GenerateProjection.scala | 9 +- .../spark/sql/catalyst/expressions/sets.scala | 129 +++++++ .../spark/sql/catalyst/trees/TreeNode.scala | 3 +- .../ExpressionEvaluationSuite.scala | 10 + .../spark/sql/execution/Aggregate.scala | 2 +- .../sql/execution/GeneratedAggregate.scala | 36 +- .../sql/execution/SparkSqlSerializer.scala | 86 +++++ .../spark/sql/execution/SparkStrategies.scala | 6 +- .../apache/spark/sql/execution/joins.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 8 +- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 8 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- ... values-0-1843b7947729b771fee3a4abd050bfdc | 1 + ...ll long-0-89b850197b326239d60a5e1d5db7c9c9 | 1 + ... + null-0-a014038c00fb81e88041ed4a8368e6f7 | 1 + ...ue long-0-77b9ed1d7ae65fa53830a3bc586856ff | 1 + ...strings-0-c68e75ec4c884b93765a466e992e391d | 1 + ...1 value-0-a4047b06a324fb5ea400c94350c9e038 | 1 + ...ng null-0-75672236a30e10dab13b9b246c5a3a1e | 1 + ...es long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 | 1 + ... values-0-c61df65af167acaf7edb174e77898f3e | 1 + ...limited-0-52b0e534c7df544258a1c59df9f816ce | 0 .../spark/sql/hive/StatisticsSuite.scala | 8 +- .../sql/hive/execution/HiveQuerySuite.scala | 65 ++++ .../sql/hive/execution/HiveSerDeSuite.scala | 11 +- .../sql/hive/execution/PruningSuite.scala | 5 +- 33 files changed, 1239 insertions(+), 34 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala create mode 100644 sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e create mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 8fc5896974438..ef1d12531f109 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -27,7 +27,8 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) - protected val exprArray = expressions.toArray + // null check is required for when Kryo invokes the no-arg constructor. + protected val exprArray = if (expressions != null) expressions.toArray else null def apply(input: Row): Row = { val outputArray = new Array[Any](exprArray.length) @@ -109,7 +110,346 @@ class JoinedRow extends Row { def apply(i: Int) = if (i < row1.size) row1(i) else row2(i - row1.size) - def isNullAt(i: Int) = apply(i) == null + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + * The `JoinedRow` class is used in many performance critical situation. Unfortunately, since there + * are multiple different types of `Rows` that could be stored as `row1` and `row2` most of the + * calls in the critical path are polymorphic. By creating special versions of this class that are + * used in only a single location of the code, we increase the chance that only a single type of + * Row will be referenced, increasing the opportunity for the JIT to play tricks. This sounds + * crazy but in benchmarks it had noticeable effects. + */ +class JoinedRow2 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow3 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow4 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow5 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) def getInt(i: Int): Int = if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index c9a63e201ef60..d68a4fabeac77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -127,7 +127,7 @@ object EmptyRow extends Row { * the array is not copied, and thus could technically be mutated after creation, this is not * allowed. */ -class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { +class GenericRow(protected[sql] val values: Array[Any]) extends Row { /** No-arg constructor for serialization. */ def this() = this(null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala new file mode 100644 index 0000000000000..75ea0e8459df8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -0,0 +1,307 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.types._ + +/** + * A parent class for mutable container objects that are reused when the values are changed, + * resulting in less garbage. These values are held by a [[SpecificMutableRow]]. + * + * The following code was roughly used to generate these objects: + * {{{ + * val types = "Int,Float,Boolean,Double,Short,Long,Byte,Any".split(",") + * types.map {tpe => + * s""" + * final class Mutable$tpe extends MutableValue { + * var value: $tpe = 0 + * def boxed = if (isNull) null else value + * def update(v: Any) = value = { + * isNull = false + * v.asInstanceOf[$tpe] + * } + * def copy() = { + * val newCopy = new Mutable$tpe + * newCopy.isNull = isNull + * newCopy.value = value + * newCopy.asInstanceOf[this.type] + * } + * }""" + * }.foreach(println) + * + * types.map { tpe => + * s""" + * override def set$tpe(ordinal: Int, value: $tpe): Unit = { + * val currentValue = values(ordinal).asInstanceOf[Mutable$tpe] + * currentValue.isNull = false + * currentValue.value = value + * } + * + * override def get$tpe(i: Int): $tpe = { + * values(i).asInstanceOf[Mutable$tpe].value + * }""" + * }.foreach(println) + * }}} + */ +abstract class MutableValue extends Serializable { + var isNull: Boolean = true + def boxed: Any + def update(v: Any) + def copy(): this.type +} + +final class MutableInt extends MutableValue { + var value: Int = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Int] + } + def copy() = { + val newCopy = new MutableInt + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableFloat extends MutableValue { + var value: Float = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Float] + } + def copy() = { + val newCopy = new MutableFloat + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableBoolean extends MutableValue { + var value: Boolean = false + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Boolean] + } + def copy() = { + val newCopy = new MutableBoolean + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableDouble extends MutableValue { + var value: Double = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Double] + } + def copy() = { + val newCopy = new MutableDouble + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableShort extends MutableValue { + var value: Short = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Short] + } + def copy() = { + val newCopy = new MutableShort + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableLong extends MutableValue { + var value: Long = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Long] + } + def copy() = { + val newCopy = new MutableLong + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableByte extends MutableValue { + var value: Byte = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Byte] + } + def copy() = { + val newCopy = new MutableByte + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableAny extends MutableValue { + var value: Any = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Any] + } + def copy() = { + val newCopy = new MutableAny + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +/** + * A row type that holds an array specialized container objects, of type [[MutableValue]], chosen + * based on the dataTypes of each column. The intent is to decrease garbage when modifying the + * values of primitive columns. + */ +final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableRow { + + def this(dataTypes: Seq[DataType]) = + this( + dataTypes.map { + case IntegerType => new MutableInt + case ByteType => new MutableByte + case FloatType => new MutableFloat + case ShortType => new MutableShort + case DoubleType => new MutableDouble + case BooleanType => new MutableBoolean + case LongType => new MutableLong + case _ => new MutableAny + }.toArray) + + def this() = this(Seq.empty) + + override def length: Int = values.length + + override def setNullAt(i: Int): Unit = { + values(i).isNull = true + } + + override def apply(i: Int): Any = values(i).boxed + + override def isNullAt(i: Int): Boolean = values(i).isNull + + override def copy(): Row = { + val newValues = new Array[MutableValue](values.length) + var i = 0 + while (i < values.length) { + newValues(i) = values(i).copy() + i += 1 + } + new SpecificMutableRow(newValues) + } + + override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + + override def iterator: Iterator[Any] = values.map(_.boxed).iterator + + def setString(ordinal: Int, value: String) = update(ordinal, value) + + def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + + override def setInt(ordinal: Int, value: Int): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableInt] + currentValue.isNull = false + currentValue.value = value + } + + override def getInt(i: Int): Int = { + values(i).asInstanceOf[MutableInt].value + } + + override def setFloat(ordinal: Int, value: Float): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableFloat] + currentValue.isNull = false + currentValue.value = value + } + + override def getFloat(i: Int): Float = { + values(i).asInstanceOf[MutableFloat].value + } + + override def setBoolean(ordinal: Int, value: Boolean): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableBoolean] + currentValue.isNull = false + currentValue.value = value + } + + override def getBoolean(i: Int): Boolean = { + values(i).asInstanceOf[MutableBoolean].value + } + + override def setDouble(ordinal: Int, value: Double): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableDouble] + currentValue.isNull = false + currentValue.value = value + } + + override def getDouble(i: Int): Double = { + values(i).asInstanceOf[MutableDouble].value + } + + override def setShort(ordinal: Int, value: Short): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableShort] + currentValue.isNull = false + currentValue.value = value + } + + override def getShort(i: Int): Short = { + values(i).asInstanceOf[MutableShort].value + } + + override def setLong(ordinal: Int, value: Long): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableLong] + currentValue.isNull = false + currentValue.value = value + } + + override def getLong(i: Int): Long = { + values(i).asInstanceOf[MutableLong].value + } + + override def setByte(ordinal: Int, value: Byte): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableByte] + currentValue.isNull = false + currentValue.value = value + } + + override def getByte(i: Int): Byte = { + values(i).asInstanceOf[MutableByte].value + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 01947273b6ccc..613b87ca98d97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -22,6 +22,7 @@ import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.util.collection.OpenHashSet abstract class AggregateExpression extends Expression { self: Product => @@ -161,13 +162,88 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new CountFunction(child, this) } -case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { +case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate { + def this() = this(null) + override def children = expressions override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) + + override def asPartial = { + val partialSet = Alias(CollectHashSet(expressions), "partialSets")() + SplitEvaluation( + CombineSetsAndCount(partialSet.toAttribute), + partialSet :: Nil) + } +} + +case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression { + def this() = this(null) + + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = ArrayType(expressions.head.dataType) + override def toString = s"AddToHashSet(${expressions.mkString(",")})" + override def newInstance() = new CollectHashSetFunction(expressions, this) +} + +case class CollectHashSetFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) + + override def update(input: Row): Unit = { + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) + } + } + + override def eval(input: Row): Any = { + seen + } +} + +case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression { + def this() = this(null) + + override def children = inputSet :: Nil + override def references = inputSet.references + override def nullable = false + override def dataType = LongType + override def toString = s"CombineAndCount($inputSet)" + override def newInstance() = new CombineSetsAndCountFunction(inputSet, this) +} + +case class CombineSetsAndCountFunction( + @transient inputSet: Expression, + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + override def update(input: Row): Unit = { + val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] + val inputIterator = inputSetEval.iterator + while (inputIterator.hasNext) { + seen.add(inputIterator.next) + } + } + + override def eval(input: Row): Any = seen.size.toLong } case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) @@ -379,17 +455,22 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } -case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) +case class CountDistinctFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) override def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.eval(input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { - seen += evaluatedExpr + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index c79c1847cedf5..8d90614e4501a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -85,3 +85,34 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } + +case class MaxOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def nullable = left.nullable && right.nullable + + override def children = left :: right :: Nil + + override def references = left.references ++ right.references + + override def dataType = left.dataType + + override def eval(input: Row): Any = { + val leftEval = left.eval(input) + val rightEval = right.eval(input) + if (leftEval == null) { + rightEval + } else if (rightEval == null) { + leftEval + } else { + val numeric = left.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]] + if (numeric.compare(leftEval, rightEval) < 0) { + rightEval + } else { + leftEval + } + } + } + + override def toString = s"MaxOf($left, $right)" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index de2d67ce82ff1..5a3f013c34579 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -26,6 +26,10 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ +// These classes are here to avoid issues with serialization and integration with quasiquotes. +class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] +class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] + /** * A base class for generators of byte code to perform expression evaluation. Includes a set of * helpers for referring to Catalyst types and building trees that perform evaluation of individual @@ -50,6 +54,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin private val curId = new java.util.concurrent.atomic.AtomicInteger() private val javaSeparator = "$" + /** + * Can be flipped on manually in the console to add (expensive) expression evaluation trace code. + */ + var debugLogging = false + /** * Generates a class for a given input expression. Called when there is not cached code * already available. @@ -71,7 +80,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin * From the Guava Docs: A Cache is similar to ConcurrentMap, but not quite the same. The most * fundamental difference is that a ConcurrentMap persists all elements that are added to it until * they are explicitly removed. A Cache on the other hand is generally configured to evict entries - * automatically, in order to constrain its memory footprint + * automatically, in order to constrain its memory footprint. Note that this cache does not use + * weak keys/values and thus does not respond to memory pressure. */ protected val cache = CacheBuilder.newBuilder() .maximumSize(1000) @@ -403,6 +413,78 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin $primitiveTerm = ${falseEval.primitiveTerm} } """.children + + case NewSet(elementType) => + q""" + val $nullTerm = false + val $primitiveTerm = new ${hashSetForType(elementType)}() + """.children + + case AddItemToSet(item, set) => + val itemEval = expressionEvaluator(item) + val setEval = expressionEvaluator(set) + + val ArrayType(elementType, _) = set.dataType + + itemEval.code ++ setEval.code ++ + q""" + if (!${itemEval.nullTerm}) { + ${setEval.primitiveTerm} + .asInstanceOf[${hashSetForType(elementType)}] + .add(${itemEval.primitiveTerm}) + } + + val $nullTerm = false + val $primitiveTerm = ${setEval.primitiveTerm} + """.children + + case CombineSets(left, right) => + val leftEval = expressionEvaluator(left) + val rightEval = expressionEvaluator(right) + + val ArrayType(elementType, _) = left.dataType + + leftEval.code ++ rightEval.code ++ + q""" + val $nullTerm = false + var $primitiveTerm: ${hashSetForType(elementType)} = null + + { + val leftSet = ${leftEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val rightSet = ${rightEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val iterator = rightSet.iterator + while (iterator.hasNext) { + leftSet.add(iterator.next()) + } + $primitiveTerm = leftSet + } + """.children + + case MaxOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $nullTerm = false + if (${eval1.primitiveTerm} > ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + } // If there was no match in the partial function above, we fall back on calling the interpreted @@ -420,7 +502,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin // Only inject debugging code if debugging is turned on. val debugCode = - if (log.isDebugEnabled) { + if (debugLogging) { val localLogger = log val localLoggerTree = reify { localLogger } q""" @@ -454,6 +536,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def accessorForType(dt: DataType) = newTermName(s"get${primitiveForType(dt)}") protected def mutatorForType(dt: DataType) = newTermName(s"set${primitiveForType(dt)}") + protected def hashSetForType(dt: DataType) = dt match { + case IntegerType => typeOf[IntegerHashSet] + case LongType => typeOf[LongHashSet] + case unsupportedType => + sys.error(s"Code generation not support for hashset of type $unsupportedType") + } + protected def primitiveForType(dt: DataType) = dt match { case IntegerType => "Int" case LongType => "Long" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 77fa02c13de30..7871a62620478 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -69,8 +69,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { ..${evaluatedExpression.code} if(${evaluatedExpression.nullTerm}) setNullAt($iLit) - else + else { + nullBits($iLit) = false $elementName = ${evaluatedExpression.primitiveTerm} + } } """.children : Seq[Tree] } @@ -106,9 +108,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { if(value == null) { setNullAt(i) } else { + nullBits(i) = false $elementName = value.asInstanceOf[${termForType(e.dataType)}] - return } + return }""" } q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" @@ -137,7 +140,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? - q"if(i == $i) { $elementName = value; return }" :: Nil + q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala new file mode 100644 index 0000000000000..e6c570b47bee2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -0,0 +1,129 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.util.collection.OpenHashSet + +/** + * Creates a new set of the specified type + */ +case class NewSet(elementType: DataType) extends LeafExpression { + type EvaluatedType = Any + + def references = Set.empty + + def nullable = false + + // We are currently only using these Expressions internally for aggregation. However, if we ever + // expose these to users we'll want to create a proper type instead of hijacking ArrayType. + def dataType = ArrayType(elementType) + + def eval(input: Row): Any = { + new OpenHashSet[Any]() + } + + override def toString = s"new Set($dataType)" +} + +/** + * Adds an item to a set. + * For performance, this expression mutates its input during evaluation. + */ +case class AddItemToSet(item: Expression, set: Expression) extends Expression { + type EvaluatedType = Any + + def children = item :: set :: Nil + + def nullable = set.nullable + + def dataType = set.dataType + + def references = (item.flatMap(_.references) ++ set.flatMap(_.references)).toSet + + def eval(input: Row): Any = { + val itemEval = item.eval(input) + val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] + + if (itemEval != null) { + if (setEval != null) { + setEval.add(itemEval) + setEval + } else { + null + } + } else { + setEval + } + } + + override def toString = s"$set += $item" +} + +/** + * Combines the elements of two sets. + * For performance, this expression mutates its left input set during evaluation. + */ +case class CombineSets(left: Expression, right: Expression) extends BinaryExpression { + type EvaluatedType = Any + + def nullable = left.nullable || right.nullable + + def dataType = left.dataType + + def symbol = "++=" + + def eval(input: Row): Any = { + val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]] + if(leftEval != null) { + val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]] + if (rightEval != null) { + val iterator = rightEval.iterator + while(iterator.hasNext) { + val rightValue = iterator.next() + leftEval.add(rightValue) + } + leftEval + } else { + null + } + } else { + null + } + } +} + +/** + * Returns the number of elements in the input set. + */ +case class CountSet(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def nullable = child.nullable + + def dataType = LongType + + def eval(input: Row): Any = { + val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]] + if (childEval != null) { + childEval.size.toLong + } + } + + override def toString = s"$child.count()" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index cd04bdf02cf84..96ce35939e2cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -280,7 +280,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { try { - val defaultCtor = getClass.getConstructors.head + // Skip no-arg constructors that are just there for kryo. + val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head if (otherCopyArgs.isEmpty) { defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 999c9fff38d60..f1df817c41362 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -136,6 +136,16 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) } + test("MaxOf") { + checkEvaluation(MaxOf(1, 2), 2) + checkEvaluation(MaxOf(2, 1), 2) + checkEvaluation(MaxOf(1L, 2L), 2L) + checkEvaluation(MaxOf(2L, 1L), 2L) + + checkEvaluation(MaxOf(Literal(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal(null, IntegerType)), 2) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 463a1d32d7fd7..be9f155253d77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -175,7 +175,7 @@ case class Aggregate( private[this] val resultProjection = new InterpretedMutableProjection( resultExpressions, computedSchema ++ namedGroups.map(_._2)) - private[this] val joinedRow = new JoinedRow + private[this] val joinedRow = new JoinedRow4 override final def hasNext: Boolean = hashTableIter.hasNext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 4a26934c49c93..31ad5e8aabb0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -103,6 +103,40 @@ case class GeneratedAggregate( updateCount :: updateSum :: Nil, result ) + + case m @ Max(expr) => + val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() + val initialValue = Literal(null, expr.dataType) + val updateMax = MaxOf(currentMax, expr) + + AggregateEvaluation( + currentMax :: Nil, + initialValue :: Nil, + updateMax :: Nil, + currentMax) + + case CollectHashSet(Seq(expr)) => + val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val initialValue = NewSet(expr.dataType) + val addToSet = AddItemToSet(expr, set) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + addToSet :: Nil, + set) + + case CombineSetsAndCount(inputSet) => + val ArrayType(inputType, _) = inputSet.dataType + val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() + val initialValue = NewSet(inputType) + val collectSets = CombineSets(set, inputSet) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + collectSets :: Nil, + CountSet(set)) } val computationSchema = computeFunctions.flatMap(_.schema) @@ -151,7 +185,7 @@ case class GeneratedAggregate( (namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq) log.info(s"Result Projection: ${resultExpressions.mkString(",")}") - val joinedRow = new JoinedRow + val joinedRow = new JoinedRow3 if (groupingExpressions.isEmpty) { // TODO: Codegening anything other than the updateProjection is probably over kill. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 34654447a5f4b..077e6ebc5f11e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -28,9 +28,13 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} + private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() @@ -41,6 +45,13 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], new HyperLogLogSerializer) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) + + // Specific hashsets must come first TODO: Move to core. + kryo.register(classOf[IntegerHashSet], new IntegerHashSetSerializer) + kryo.register(classOf[LongHashSet], new LongHashSetSerializer) + kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], + new OpenHashSetSerializer) + kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) new AllScalaRegistrar().apply(kryo) @@ -109,3 +120,78 @@ private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { HyperLogLog.Builder.build(bytes) } } + +private[sql] class OpenHashSetSerializer extends Serializer[OpenHashSet[_]] { + def write(kryo: Kryo, output: Output, hs: OpenHashSet[_]) { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val row = iterator.next() + rowSerializer.write(kryo, output, row.asInstanceOf[GenericRow].values) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[OpenHashSet[_]]): OpenHashSet[_] = { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + val numItems = input.readInt() + val set = new OpenHashSet[Any](numItems + 1) + var i = 0 + while (i < numItems) { + val row = + new GenericRow(rowSerializer.read( + kryo, + input, + classOf[Array[Any]].asInstanceOf[Class[Any]]).asInstanceOf[Array[Any]]) + set.add(row) + i += 1 + } + set + } +} + +private[sql] class IntegerHashSetSerializer extends Serializer[IntegerHashSet] { + def write(kryo: Kryo, output: Output, hs: IntegerHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value: Int = iterator.next() + output.writeInt(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[IntegerHashSet]): IntegerHashSet = { + val numItems = input.readInt() + val set = new IntegerHashSet + var i = 0 + while (i < numItems) { + val value = input.readInt() + set.add(value) + i += 1 + } + set + } +} + +private[sql] class LongHashSetSerializer extends Serializer[LongHashSet] { + def write(kryo: Kryo, output: Output, hs: LongHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value = iterator.next() + output.writeLong(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[LongHashSet]): LongHashSet = { + val numItems = input.readInt() + val set = new LongHashSet + var i = 0 + while (i < numItems) { + val value = input.readLong() + set.add(value) + i += 1 + } + set + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f0c958fdb537f..517b77804ae2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ @@ -148,7 +149,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { - case _: Sum | _: Count => false + case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + // The generated set implementation is pretty limited ATM. + case CollectHashSet(exprs) if exprs.size == 1 && + Seq(IntegerType, LongType).contains(exprs.head.dataType) => false case _ => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index b08f9aacc1fcb..2890a563bed48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -92,7 +92,7 @@ trait HashJoin { private[this] var currentMatchPosition: Int = -1 // Mutable per row objects. - private[this] val joinRow = new JoinedRow + private[this] val joinRow = new JoinedRow2 private[this] val joinKeys = streamSideKeyGenerator() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 0a3b59cbc233a..ef4526ec03439 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -23,7 +23,7 @@ import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType /** @@ -278,14 +278,14 @@ private[parquet] class CatalystGroupConverter( */ private[parquet] class CatalystPrimitiveRowConverter( protected[parquet] val schema: Array[FieldType], - protected[parquet] var current: ParquetRelation.RowType) + protected[parquet] var current: MutableRow) extends CatalystConverter { // This constructor is used for the root converter only def this(attributes: Array[Attribute]) = this( attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), - new ParquetRelation.RowType(attributes.length)) + new SpecificMutableRow(attributes.map(_.dataType))) protected [parquet] val converters: Array[Converter] = schema.zipWithIndex.map { @@ -299,7 +299,7 @@ private[parquet] class CatalystPrimitiveRowConverter( override val parent = null // Should be only called in root group converter! - override def getCurrentRecord: ParquetRelation.RowType = current + override def getCurrentRecord: Row = current override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index f6cfab736d98a..a5a5d139a65cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -139,7 +139,7 @@ case class ParquetTableScan( partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) new Iterator[Row] { - private[this] val joinedRow = new JoinedRow(Row(partitionRowValues:_*), null) + private[this] val joinedRow = new JoinedRow5(Row(partitionRowValues:_*), null) def hasNext = iter.hasNext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 76b1724471442..37d64f0de7bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -45,16 +45,16 @@ class PlannerSuite extends FunSuite { assert(aggregations.size === 2) } - test("count distinct is not partially aggregated") { + test("count distinct is partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } - test("mixed aggregates are not partially aggregated") { + test("mixed aggregates are partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 3b371211e14cd..6571c35499ef4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -265,9 +265,9 @@ private[hive] case class MetastoreRelation // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - @transient lazy val hiveQlTable = new Table(table) + @transient val hiveQlTable = new Table(table) - def hiveQlPartitions = partitions.map { p => + @transient val hiveQlPartitions = partitions.map { p => new Partition(hiveQlTable, p) } diff --git a/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 7c82964b5ecdc..8d6ca9939a730 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.scalatest.BeforeAndAfterAll + import scala.reflect.ClassTag @@ -26,7 +28,9 @@ import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -class StatisticsSuite extends QueryTest { +class StatisticsSuite extends QueryTest with BeforeAndAfterAll { + TestHive.reset() + TestHive.cacheTables = false test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { @@ -126,7 +130,7 @@ class StatisticsSuite extends QueryTest { val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(sizes.size === 1) + assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index fdb2f41f5a5b6..26e4ec6e6dcce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -32,6 +32,71 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("count distinct 0 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 0) table + """.stripMargin) + + createQueryTest("count distinct 1 value strings", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 1 UNION ALL + | SELECT 'b' AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 2 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values including null", + """ + |SELECT COUNT(DISTINCT a, 1) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 2L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + createQueryTest("null case", "SELECT case when(true) then 1 else null end FROM src LIMIT 1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index df9bae96494d5..8bc72384a64ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -17,10 +17,19 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.hive.test.TestHive + /** * A set of tests that validates support for Hive SerDe. */ -class HiveSerDeSuite extends HiveComparisonTest { +class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { + + override def beforeAll() = { + TestHive.cacheTables = false + } + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 1a6dbc0ce0c0d..8275e2d3bcce3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ @@ -25,9 +27,10 @@ import scala.collection.JavaConversions._ /** * A set of test cases that validate partition and column pruning. */ -class PruningSuite extends HiveComparisonTest { +class PruningSuite extends HiveComparisonTest with BeforeAndAfter { // MINOR HACK: You must run a query before calling reset the first time. TestHive.sql("SHOW TABLES") + TestHive.cacheTables = false // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset // the environment to ensure all referenced tables in this suites are not cached in-memory. From e23f0bc0177a83dfee3f5579ae6eb12033ae5f90 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 23 Aug 2014 16:21:08 -0700 Subject: [PATCH 087/192] [SPARK-2967][SQL] Follow-up: Also copy hash expressions in sort based shuffle fix. Follow-up to #2066 Author: Michael Armbrust Closes #2072 from marmbrus/sortShuffle and squashes the following commits: 2ff8114 [Michael Armbrust] Fix bug (cherry picked from commit 3519b5e8e55b4530d7f7c0bcab254f863dbfa814) Signed-off-by: Michael Armbrust --- .../scala/org/apache/spark/sql/execution/Exchange.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 09c34b7059fc3..4802e40595807 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -46,12 +46,15 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().mapPartitions { iter => - @transient val hashExpressions = - newMutableProjection(expressions, child.output)() - if (sortBasedShuffleOn) { + @transient val hashExpressions = + newProjection(expressions, child.output) + iter.map(r => (hashExpressions(r), r.copy())) } else { + @transient val hashExpressions = + newMutableProjection(expressions, child.output)() + val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } From ce14cd11f099e46532074bc23a7ffb1bad0969e6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 24 Aug 2014 09:43:44 -0700 Subject: [PATCH 088/192] [SPARK-3192] Some scripts have 2 space indentation but other scripts have 4 space indentation. Author: Kousuke Saruta Closes #2104 from sarutak/SPARK-3192 and squashes the following commits: db78419 [Kousuke Saruta] Modified indentation of spark-shell (cherry picked from commit ded6796bf54f5c005b27135d7dec19634038a1c6) Signed-off-by: Patrick Wendell --- bin/spark-shell | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 8b7ccd7439551..0ab4e14f5b744 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -22,7 +22,7 @@ cygwin=false case "`uname`" in - CYGWIN*) cygwin=true;; + CYGWIN*) cygwin=true;; esac # Enter posix mode for bash @@ -32,9 +32,9 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" function usage() { - echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -46,20 +46,20 @@ SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" function main() { - if $cygwin; then - # Workaround for issue involving JLine and Cygwin - # (see http://sourceforge.net/p/jline/bugs/40/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "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_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" - stty icanon echo > /dev/null 2>&1 - else - export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" - fi + if $cygwin; then + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "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_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + stty icanon echo > /dev/null 2>&1 + else + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + fi } # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in From a4db81a55f266f904052525aa290b7ffcf9a613c Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sun, 24 Aug 2014 17:33:33 -0700 Subject: [PATCH 089/192] [SPARK-2841][MLlib] Documentation for feature transformations Documentation for newly added feature transformations: 1. TF-IDF 2. StandardScaler 3. Normalizer Author: DB Tsai Closes #2068 from dbtsai/transformer-documentation and squashes the following commits: 109f324 [DB Tsai] address feedback (cherry picked from commit 572952ae615895efaaabcd509d582262000c0852) Signed-off-by: Xiangrui Meng --- docs/mllib-feature-extraction.md | 109 ++++++++++++++++++++++++++++++- 1 file changed, 107 insertions(+), 2 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 2031b96235ee9..44f0f76220b6e 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -1,7 +1,7 @@ --- layout: global -title: Feature Extraction - MLlib -displayTitle: MLlib - Feature Extraction +title: Feature Extraction and Transformation - MLlib +displayTitle: MLlib - Feature Extraction and Transformation --- * Table of contents @@ -148,3 +148,108 @@ for((synonym, cosineSimilarity) <- synonyms) { {% endhighlight %}
    + +## StandardScaler + +Standardizes features by scaling to unit variance and/or removing the mean using column summary +statistics on the samples in the training set. This is a very common pre-processing step. + +For example, RBF kernel of Support Vector Machines or the L1 and L2 regularized linear models +typically work better when all features have unit variance and/or zero mean. + +Standardization can improve the convergence rate during the optimization process, and also prevents +against features with very large variances exerting an overly large influence during model training. + +### Model Fitting + +[`StandardScaler`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) has the +following parameters in the constructor: + +* `withMean` False by default. Centers the data with mean before scaling. It will build a dense +output, so this does not work on sparse input and will raise an exception. +* `withStd` True by default. Scales the data to unit variance. + +We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) method in +`StandardScaler` which can take an input of `RDD[Vector]`, learn the summary statistics, and then +return a model which can transform the input dataset into unit variance and/or zero mean features +depending how we configure the `StandardScaler`. + +This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the standardization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the variance of a feature is zero, it will return default `0.0` value in the `Vector` +for that feature. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and standardize the features +so that the new features have unit variance and/or zero mean. + +
    +
    +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.StandardScaler +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val scaler1 = new StandardScaler().fit(data.map(x => x.features)) +val scaler2 = new StandardScaler(withMean = true, withStd = true).fit(data.map(x => x.features)) + +// data1 will be unit variance. +val data1 = data.map(x => (x.label, scaler1.transform(x.features))) + +// Without converting the features into dense vectors, transformation with zero mean will raise +// exception on sparse vector. +// data2 will be unit variance and zero mean. +val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) +{% endhighlight %} +
    +
    + +## Normalizer + +Normalizer scales individual samples to have unit $L^p$ norm. This is a common operation for text +classification or clustering. For example, the dot product of two $L^2$ normalized TF-IDF vectors +is the cosine similarity of the vectors. + +[`Normalizer`](api/scala/index.html#org.apache.spark.mllib.feature.Normalizer) has the following +parameter in the constructor: + +* `p` Normalization in $L^p$ space, $p = 2$ by default. + +`Normalizer` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the normalization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the norm of the input is zero, it will return the input vector. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and normalizes the features +with $L^2$ norm, and $L^\infty$ norm. + +
    +
    +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Normalizer +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val normalizer1 = new Normalizer() +val normalizer2 = new Normalizer(p = Double.PositiveInfinity) + +// Each sample in data1 will be normalized using $L^2$ norm. +val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) + +// Each sample in data2 will be normalized using $L^\infty$ norm. +val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) +{% endhighlight %} +
    +
    From 749bddc85e76e0d1ded8d79058819335bd580741 Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Sun, 24 Aug 2014 17:35:54 -0700 Subject: [PATCH 090/192] [MLlib][SPARK-2997] Update SVD documentation to reflect roughly square Update the documentation to reflect the fact we can handle roughly square matrices. Author: Reza Zadeh Closes #2070 from rezazadeh/svddocs and squashes the following commits: 826b8fe [Reza Zadeh] left singular vectors 3f34fc6 [Reza Zadeh] PCA is still TS 7ffa2aa [Reza Zadeh] better title aeaf39d [Reza Zadeh] More docs 788ed13 [Reza Zadeh] add computational cost explanation 6429c59 [Reza Zadeh] Add link to rowmatrix docs 1eeab8b [Reza Zadeh] Update SVD documentation to reflect roughly square (cherry picked from commit b1b20301b3a1b35564d61e58eb5964d5ad5e4d7d) Signed-off-by: Xiangrui Meng --- docs/mllib-dimensionality-reduction.md | 29 ++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 065d646496131..9f2cf6d48ec75 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on tall-and-skinny matrices. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -39,8 +39,26 @@ If we keep the top $k$ singular values, then the dimensions of the resulting low * `$\Sigma$`: `$k \times k$`, * `$V$`: `$n \times k$`. -MLlib provides SVD functionality to row-oriented matrices that have only a few columns, -say, less than $1000$, but many rows, i.e., *tall-and-skinny* matrices. +### Performance +We assume $n$ is smaller than $m$. The singular values and the right singular vectors are derived +from the eigenvalues and the eigenvectors of the Gramian matrix $A^T A$. The matrix +storing the left singular vectors $U$, is computed via matrix multiplication as +$U = A (V S^{-1})$, if requested by the user via the computeU parameter. +The actual method to use is determined automatically based on the computational cost: + +* If $n$ is small ($n < 100$) or $k$ is large compared with $n$ ($k > n / 2$), we compute the Gramian matrix +first and then compute its top eigenvalues and eigenvectors locally on the driver. +This requires a single pass with $O(n^2)$ storage on each executor and on the driver, and +$O(n^2 k)$ time on the driver. +* Otherwise, we compute $(A^T A) v$ in a distributive way and send it to +ARPACK to +compute $(A^T A)$'s top eigenvalues and eigenvectors on the driver node. This requires $O(k)$ +passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. + +### SVD Example + +MLlib provides SVD functionality to row-oriented matrices, provided in the +RowMatrix class.
    @@ -124,9 +142,8 @@ MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format.
    -The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. -The number of columns should be small, e.g, less than 1000. {% highlight scala %} import org.apache.spark.mllib.linalg.Matrix @@ -144,7 +161,7 @@ val projected: RowMatrix = mat.multiply(pc)
    -The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. The number of columns should be small, e.g, less than 1000. From b82da3d6924a5bd2139434ab05c2fd44914fda45 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 24 Aug 2014 21:16:05 -0700 Subject: [PATCH 091/192] [SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId() RDD.zipWithIndex() 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 method needs to trigger a spark job when this RDD contains more than one partitions. >>> sc.parallelize(range(4), 2).zipWithIndex().collect() [(0, 0), (1, 1), (2, 2), (3, 3)] RDD.zipWithUniqueId() 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 L{zipWithIndex} >>> sc.parallelize(range(4), 2).zipWithUniqueId().collect() [(0, 0), (2, 1), (1, 2), (3, 3)] Author: Davies Liu Closes #2092 from davies/zipWith and squashes the following commits: cebe5bf [Davies Liu] improve test cases, reverse the order of index 0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId() (cherry picked from commit fb0db772421b6902b80137bf769db3b418ab2ccf) Signed-off-by: Josh Rosen --- python/pyspark/rdd.py | 47 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3eefc878d274e..f0706d846d014 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1715,6 +1715,53 @@ def batch_as(rdd, batchSize): other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) + def zipWithIndex(self): + """ + 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 method needs to trigger a spark job when this RDD contains + more than one partitions. + + >>> sc.parallelize(["a", "b", "c", "d"], 3).zipWithIndex().collect() + [('a', 0), ('b', 1), ('c', 2), ('d', 3)] + """ + starts = [0] + if self.getNumPartitions() > 1: + nums = self.mapPartitions(lambda it: [sum(1 for i in it)]).collect() + for i in range(len(nums) - 1): + starts.append(starts[-1] + nums[i]) + + def func(k, it): + for i, v in enumerate(it, starts[k]): + yield v, i + + return self.mapPartitionsWithIndex(func) + + def zipWithUniqueId(self): + """ + 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 + L{zipWithIndex} + + >>> sc.parallelize(["a", "b", "c", "d", "e"], 3).zipWithUniqueId().collect() + [('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)] + """ + n = self.getNumPartitions() + + def func(k, it): + for i, v in enumerate(it): + yield v, i * n + k + + return self.mapPartitionsWithIndex(func) + def name(self): """ Return the name of this RDD. From 69a17f119758e786ef080cfbf52d484334c8d9d9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 25 Aug 2014 12:30:02 -0700 Subject: [PATCH 092/192] [SPARK-2495][MLLIB] make KMeans constructor public to re-construct k-means models freeman-lab Author: Xiangrui Meng Closes #2112 from mengxr/public-constructors and squashes the following commits: 18d53a9 [Xiangrui Meng] make KMeans constructor public (cherry picked from commit 220f413686ae922bd11776576bf37610cce92c23) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/mllib/clustering/KMeansModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 5823cb6e52e7f..12a3d91cd31a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -25,7 +25,7 @@ import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length From ff616fd7b4b56c34bd473f85fab3524b842da404 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 25 Aug 2014 13:29:07 -0700 Subject: [PATCH 093/192] SPARK-2798 [BUILD] Correct several small errors in Flume module pom.xml files (EDIT) Since the scalatest issue was since resolved, this is now about a few small problems in the Flume Sink `pom.xml` - `scalatest` is not declared as a test-scope dependency - Its Avro version doesn't match the rest of the build - Its Flume version is not synced with the other Flume module - The other Flume module declares its dependency on Flume Sink slightly incorrectly, hard-coding the Scala 2.10 version - It depends on Scala Lang directly, which it shouldn't Author: Sean Owen Closes #1726 from srowen/SPARK-2798 and squashes the following commits: a46e2c6 [Sean Owen] scalatest to test scope, harmonize Avro and Flume versions, remove direct Scala dependency, fix '2.10' in Flume dependency (cherry picked from commit cd30db566a327ddf63cd242c758e46ce2d9479df) Signed-off-by: Tathagata Das --- external/flume-sink/pom.xml | 15 ++++++--------- external/flume/pom.xml | 12 ++++++------ pom.xml | 1 + 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4373cb058e7c0..a297459f1658e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -25,19 +25,20 @@ ../../pom.xml + org.apache.spark spark-streaming-flume-sink_2.10 streaming-flume-sink - jar Spark Project External Flume Sink http://spark.apache.org/ + org.apache.flume flume-ng-sdk - 1.4.0 + ${flume.version} io.netty @@ -52,7 +53,7 @@ org.apache.flume flume-ng-core - 1.4.0 + ${flume.version} io.netty @@ -62,11 +63,7 @@ org.apache.thrift libthrift - - - - org.scala-lang - scala-library + org.scalatest @@ -97,7 +94,7 @@ org.apache.avro avro-maven-plugin - 1.7.3 + ${avro.version} ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b127136e3f5a0..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -40,6 +40,11 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming-flume-sink_${scala.binary.version} + ${project.version} + org.apache.spark spark-streaming_${scala.binary.version} @@ -50,7 +55,7 @@ org.apache.flume flume-ng-sdk - 1.4.0 + ${flume.version} io.netty @@ -82,11 +87,6 @@ junit-interface test - - org.apache.spark - spark-streaming-flume-sink_2.10 - ${project.version} - target/scala-${scala.binary.version}/classes diff --git a/pom.xml b/pom.xml index bc3aa060e9dfc..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -124,6 +124,7 @@ 2.4.1 ${hadoop.version} 0.94.6 + 1.4.0 3.4.5 0.12.0 1.4.3 From d892062cca16bd9d977e1cf51723135a481edf57 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 25 Aug 2014 14:55:20 -0700 Subject: [PATCH 094/192] [FIX] fix error message in sendMessageReliably rxin Author: Xiangrui Meng Closes #2120 from mengxr/sendMessageReliably and squashes the following commits: b14400c [Xiangrui Meng] fix error message in sendMessageReliably (cherry picked from commit fd8ace2d9a796f69ce34ad202907008cd6e4d274) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/network/ConnectionManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index e5e1e72cd912b..578d806263006 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -851,8 +851,8 @@ private[spark] class ConnectionManager( messageStatuses.synchronized { messageStatuses.remove(message.id).foreach ( s => { promise.failure( - new IOException(s"sendMessageReliably failed because ack " + - "was not received within ${ackTimeout} sec")) + new IOException("sendMessageReliably failed because ack " + + s"was not received within $ackTimeout sec")) }) } } From 8d33a6d3de9184ee33ebe5f30fef6a1fda281e9d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Aug 2014 14:56:51 -0700 Subject: [PATCH 095/192] Fixed a typo in docs/running-on-mesos.md It should be `spark-env.sh` rather than `spark.env.sh`. Author: Cheng Lian Closes #2119 from liancheng/fix-mesos-doc and squashes the following commits: f360548 [Cheng Lian] Fixed a typo in docs/running-on-mesos.md (cherry picked from commit 805fec845b7aa8b4763e3e0e34bec6c3872469f4) Signed-off-by: Josh Rosen --- docs/running-on-mesos.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index bd046cfc1837d..9998dddc652a6 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -107,7 +107,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: -1. In `spark.env.sh` set some environment variables: +1. In `spark-env.sh` set some environment variables: * `export MESOS_NATIVE_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of From 19b01d6f79f2919257fcd14524bc8267c57eb3d9 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 25 Aug 2014 16:27:00 -0700 Subject: [PATCH 096/192] [SPARK-3204][SQL] MaxOf would be foldable if both left and right are foldable. Author: Takuya UESHIN Closes #2116 from ueshin/issues/SPARK-3204 and squashes the following commits: 7d9b107 [Takuya UESHIN] Make MaxOf foldable if both left and right are foldable. (cherry picked from commit d299e2bf2f6733a6267b7ce85e2b288608b17db3) Signed-off-by: Michael Armbrust --- .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 8d90614e4501a..5f8b6ae10f0c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -89,6 +89,8 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any + override def foldable = left.foldable && right.foldable + override def nullable = left.nullable && right.nullable override def children = left :: right :: Nil From 292f28d4f7cbfdb8b90809926a6d69df7ed817e7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Aug 2014 16:29:59 -0700 Subject: [PATCH 097/192] [SPARK-2929][SQL] Refactored Thrift server and CLI suites Removed most hard coded timeout, timing assumptions and all `Thread.sleep`. Simplified IPC and synchronization with `scala.sys.process` and future/promise so that the test suites can run more robustly and faster. Author: Cheng Lian Closes #1856 from liancheng/thriftserver-tests and squashes the following commits: 2d914ca [Cheng Lian] Minor refactoring 0e12e71 [Cheng Lian] Cleaned up test output 0ee921d [Cheng Lian] Refactored Thrift server and CLI suites (cherry picked from commit cae9414d3805c6cf00eab6a6144d8f90cd0212f8) Signed-off-by: Michael Armbrust --- .../sql/hive/thriftserver/CliSuite.scala | 121 +++++++--- .../thriftserver/HiveThriftServer2Suite.scala | 212 ++++++++++-------- .../sql/hive/thriftserver/TestUtils.scala | 108 --------- 3 files changed, 217 insertions(+), 224 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 70bea1ed80fda..3475c2c9db080 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -18,41 +18,112 @@ package org.apache.spark.sql.hive.thriftserver -import java.io.{BufferedReader, InputStreamReader, PrintWriter} +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} + +import java.io._ +import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { + def runCliWithin( + timeout: FiniteDuration, + extraArgs: Seq[String] = Seq.empty)( + queriesAndExpectedAnswers: (String, String)*) { + + val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) - override def beforeAll() { - val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" - val commands = - s"""../../bin/spark-sql + val command = { + val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true" + s"""$cliScript | --master local | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH - """.stripMargin.split("\\s+") - - val pb = new ProcessBuilder(commands: _*) - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "spark-sql>") + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + """.stripMargin.split("\\s+").toSeq ++ extraArgs + } + + // AtomicInteger is needed because stderr and stdout of the forked process are handled in + // different threads. + val next = new AtomicInteger(0) + val foundAllExpectedAnswers = Promise.apply[Unit]() + val queryStream = new ByteArrayInputStream(queries.mkString("\n").getBytes) + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains(expectedAnswers(next.get()))) { + if (next.incrementAndGet() == expectedAnswers.size) { + foundAllExpectedAnswers.trySuccess(()) + } + } + } + + // Searching expected output line from both stdout and stderr of the CLI process + val process = (Process(command) #< queryStream).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL CLI process exit value: $exitValue") + } + + try { + Await.result(foundAllExpectedAnswers.future, timeout) + } catch { case cause: Throwable => + logError( + s""" + |======================= + |CliSuite failure output + |======================= + |Spark SQL CLI command line: ${command.mkString(" ")} + | + |Executed query ${next.get()} "${queries(next.get())}", + |But failed to capture expected output "${expectedAnswers(next.get())}" within $timeout. + | + |${buffer.mkString("\n")} + |=========================== + |End CliSuite failure output + |=========================== + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - override def afterAll() { - process.destroy() - process.waitFor() + test("Simple commands") { + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + runCliWithin(1.minute)( + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "hive_test", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;" + -> "OK", + "CACHE TABLE hive_test;" + -> "Time taken: ", + "SELECT COUNT(*) FROM hive_test;" + -> "5", + "DROP TABLE hive_test" + -> "Time taken: " + ) } - test("simple commands") { - val dataFilePath = getDataFile("data/files/small_kv.txt") - executeQuery("create table hive_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") - executeQuery("cache table hive_test1", "Time taken") + test("Single command with -e") { + runCliWithin(1.minute, Seq("-e", "SHOW TABLES;"))("" -> "OK") } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 326b0a7275b34..38977ff162097 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -17,32 +17,32 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} -import java.io.{BufferedReader, InputStreamReader} +import java.io.File import java.net.ServerSocket -import java.sql.{Connection, DriverManager, Statement} +import java.sql.{DriverManager, Statement} +import java.util.concurrent.TimeoutException import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.apache.hive.jdbc.HiveDriver +import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** - * Test for the HiveThriftServer2 using JDBC. + * Tests for the HiveThriftServer2 using JDBC. */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { +class HiveThriftServer2Suite extends FunSuite with Logging { + Class.forName(classOf[HiveDriver].getCanonicalName) - val WAREHOUSE_PATH = getTempFilePath("warehouse") - val METASTORE_PATH = getTempFilePath("metastore") - - val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" - val TABLE = "test" - val HOST = "localhost" - val PORT = { + private val listeningHost = "localhost" + private val listeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. val socket = new ServerSocket(0) @@ -51,96 +51,126 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt port } - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } + private val warehousePath = getTempFilePath("warehouse") + private val metastorePath = getTempFilePath("metastore") + private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - override def afterAll() { stopServer() } + def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { + val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Hive Thrift server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" val command = - s"""../../sbin/start-thriftserver.sh + s"""$serverScript | --master local - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT - """.stripMargin.split("\\s+") - - val pb = new ProcessBuilder(command ++ args: _*) - val environment = pb.environment() - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on", 300000) - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - print(stdout) - print(stderr) - Thread.sleep(50) + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$listeningHost + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort + """.stripMargin.split("\\s+").toSeq + + val serverStarted = Promise[Unit]() + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains("ThriftBinaryCLIService listening on")) { + serverStarted.success(()) } } - } - private def stopServer() { - process.destroy() - process.waitFor() + val process = Process(command).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL Thrift server process exit value: $exitValue") + } + + val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" + val user = System.getProperty("user.name") + + try { + Await.result(serverStarted.future, timeout) + + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() + + try { + f(statement) + } finally { + statement.close() + connection.close() + } + } catch { + case cause: Exception => + cause match { + case _: TimeoutException => + logError(s"Failed to start Hive Thrift server within $timeout", cause) + case _ => + } + logError( + s""" + |===================================== + |HiveThriftServer2Suite failure output + |===================================== + |HiveThriftServer2 command line: ${command.mkString(" ")} + |JDBC URI: $jdbcUri + |User: $user + | + |${buffer.mkString("\n")} + |========================================= + |End HiveThriftServer2Suite failure output + |========================================= + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - test("test query execution against a Hive Thrift server") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test") - stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key INT, val STRING)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached AS SELECT * FROM test LIMIT 4") - stmt.execute("CACHE TABLE test_cached") - - var rs = stmt.executeQuery("SELECT COUNT(*) FROM test") - rs.next() - assert(rs.getInt(1) === 5) - - rs = stmt.executeQuery("SELECT COUNT(*) FROM test_cached") - rs.next() - assert(rs.getInt(1) === 4) - - stmt.close() + test("Test JDBC query execution") { + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + val queries = Seq( + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } } test("SPARK-3004 regression: result set containing NULL") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv_with_null.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test_null") - stmt.execute("CREATE TABLE test_null(key INT, val STRING)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") - - val rs = stmt.executeQuery("SELECT * FROM test_null WHERE key IS NULL") - var count = 0 - while (rs.next()) { - count += 1 - } - assert(count === 5) + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource( + "data/files/small_kv_with_null.txt") - stmt.close() - } + val queries = Seq( + "DROP TABLE IF EXISTS test_null", + "CREATE TABLE test_null(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + + (0 until 5).foreach { _ => + resultSet.next() + assert(resultSet.getInt(1) === 0) + assert(resultSet.wasNull()) + } - def createStatement(): Statement = getConnection.createStatement() + assert(!resultSet.next()) + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -object TestUtils { - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - -trait TestUtils { - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Read stdout output and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } - - protected def getDataFile(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(name) - } -} From f8ac8ed7f88d2ee976b38d4a156f64efb3740650 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 25 Aug 2014 17:43:56 -0700 Subject: [PATCH 098/192] [SPARK-3058] [SQL] Support EXTENDED for EXPLAIN Provide `extended` keyword support for `explain` command in SQL. e.g. ``` explain extended select key as a1, value as a2 from src where key=1; == Parsed Logical Plan == Project ['key AS a1#3,'value AS a2#4] Filter ('key = 1) UnresolvedRelation None, src, None == Analyzed Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = CAST(1, DoubleType)) MetastoreRelation default, src, None == Optimized Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) MetastoreRelation default, src, None == Physical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) HiveTableScan [key#8,value#9], (MetastoreRelation default, src, None), None Code Generation: false == RDD == (2) MappedRDD[14] at map at HiveContext.scala:350 MapPartitionsRDD[13] at mapPartitions at basicOperators.scala:42 MapPartitionsRDD[12] at mapPartitions at basicOperators.scala:57 MapPartitionsRDD[11] at mapPartitions at TableReader.scala:112 MappedRDD[10] at map at TableReader.scala:240 HadoopRDD[9] at HadoopRDD at TableReader.scala:230 ``` It's the sub task of #1847. But can go without any dependency. Author: Cheng Hao Closes #1962 from chenghao-intel/explain_extended and squashes the following commits: 295db74 [Cheng Hao] Fix bug in printing the simple execution plan 48bc989 [Cheng Hao] Support EXTENDED for EXPLAIN (cherry picked from commit 156eb3966176de02ec3ec90ae10e50a7ebfbbf4f) Signed-off-by: Michael Armbrust --- .../sql/catalyst/plans/logical/commands.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 13 +++-- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../apache/spark/sql/execution/commands.scala | 10 ++-- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../sql/hive/execution/HiveExplainSuite.scala | 54 +++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 8 files changed, 78 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 481a5a4f212b2..a01809c1fc5e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman * Returned by a parser when the users only wants to see what query plan would be executed, without * actually performing the execution. */ -case class ExplainCommand(plan: LogicalPlan) extends Command { +case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends Command { override def output = Seq(AttributeReference("plan", StringType, nullable = false)()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index af9f7c62a1d25..8a9f4deb6a19e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -408,10 +408,18 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = stringOrError(executedPlan) + def simpleString: String = + s"""== Physical Plan == + |${stringOrError(executedPlan)} + """ override def toString: String = - s"""== Logical Plan == + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) + // however, the `toRdd` will cause the real execution, which is not what we want. + // We need to think about how to avoid the side effect. + s"""== Parsed Logical Plan == + |${stringOrError(logical)} + |== Analyzed Logical Plan == |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -419,7 +427,6 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} |Code Generation: ${executedPlan.codegenEnabled} |== RDD == - |${stringOrError(toRdd.toDebugString)} """.stripMargin.trim } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 517b77804ae2c..8dacb84c8a17e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -301,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.SetCommand(key, value) => Seq(execution.SetCommand(key, value, plan.output)(context)) - case logical.ExplainCommand(logicalPlan) => - Seq(execution.ExplainCommand(logicalPlan, plan.output)(context)) + case logical.ExplainCommand(logicalPlan, extended) => + Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheCommand(tableName, cache) => Seq(execution.CacheCommand(tableName, cache)(context)) case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 38f37564f1788..031b695169cea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -108,15 +108,19 @@ case class SetCommand( */ @DeveloperApi case class ExplainCommand( - logicalPlan: LogicalPlan, output: Seq[Attribute])( + logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)( @transient context: SQLContext) extends LeafNode with Command { // Run through the optimizer to generate the physical plan. override protected[sql] lazy val sideEffectResult: Seq[String] = try { - "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n") + // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. + val queryExecution = context.executePlan(logicalPlan) + val outputString = if (extended) queryExecution.toString else queryExecution.simpleString + + outputString.split("\n") } catch { case cause: TreeNodeException[_] => - "Error occurred during query planning: " +: cause.getMessage.split("\n") + ("Error occurred during query planning: \n" + cause.getMessage).split("\n") } def execute(): RDD[Row] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 29baefe714c64..d9b2bc7348ad2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -424,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { logical match { case _: NativeCommand => "" case _: SetCommand => "" - case _ => executedPlan.toString + case _ => super.simpleString } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 1d9ba1b24a7a4..5da6e8df03aee 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -409,10 +409,9 @@ private[hive] object HiveQl { ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. - val Some(query) :: _ :: _ :: Nil = + val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) - // TODO: support EXTENDED? - ExplainCommand(nodeToPlan(query)) + ExplainCommand(nodeToPlan(query), extended != None) case Token("TOK_DESCTABLE", describeArgs) => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala new file mode 100644 index 0000000000000..4ed58f4be1167 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.Row + +/** + * A set of tests that validates support for Hive Explain command. + */ +class HiveExplainSuite extends QueryTest { + private def check(sqlCmd: String, exists: Boolean, keywords: String*) { + val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)") + } + } + } + + test("explain extended command") { + check(" explain select * from src where key=123 ", true, + "== Physical Plan ==") + check(" explain select * from src where key=123 ", false, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==") + check(" explain extended select * from src where key=123 ", true, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", + "== Physical Plan ==", + "Code Generation", "== RDD ==") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 26e4ec6e6dcce..6d925e56e6838 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -327,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.size > 1 && explanation.head.startsWith("Physical execution plan") + explanation.exists(_ == "== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { From 957b356576caa2ab38d1e758c2d3190421894557 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 25 Aug 2014 17:46:43 -0700 Subject: [PATCH 099/192] [SQL] logWarning should be logInfo in getResultSetSchema Author: wangfei Closes #1939 from scwf/patch-5 and squashes the following commits: f952d10 [wangfei] [SQL] logWarning should be logInfo in getResultSetSchema (cherry picked from commit 507a1b520063ad3e10b909767d9e3fd72d24415b) Signed-off-by: Michael Armbrust --- .../sql/hive/thriftserver/server/SparkSQLOperationManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 699a1103f3248..6eccb1ba6d4dc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -151,7 +151,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { From b5dc9b43bcdcbdb5ffddbda6235443f3d7411b7a Mon Sep 17 00:00:00 2001 From: Chia-Yung Su Date: Mon, 25 Aug 2014 18:20:19 -0700 Subject: [PATCH 100/192] [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile fix compile error on hadoop 0.23 for the pull request #1924. Author: Chia-Yung Su Closes #1959 from joesu/bugfix-spark3011 and squashes the following commits: be30793 [Chia-Yung Su] remove .* and _* except _metadata 8fe2398 [Chia-Yung Su] add note to explain 40ea9bd [Chia-Yung Su] fix hadoop-0.23 compile error c7e44f2 [Chia-Yung Su] match syntax f8fc32a [Chia-Yung Su] filter out tmp dir (cherry picked from commit 4243bb6634aca5b9ddf6d42778aa7b4866ce6256) Signed-off-by: Michael Armbrust --- .../main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index c79a9ac2dad81..af8cd0a73b674 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -378,7 +378,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val children = fs.listStatus(path).filterNot { status => val name = status.getPath.getName - name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME + (name(0) == '.' || name(0) == '_') && name != ParquetFileWriter.PARQUET_METADATA_FILE } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row From 4d6a0e920974a5d50348ba9f7377b48e43c2da16 Mon Sep 17 00:00:00 2001 From: witgo Date: Mon, 25 Aug 2014 19:22:27 -0700 Subject: [PATCH 101/192] SPARK-2481: The environment variables SPARK_HISTORY_OPTS is covered in spark-env.sh Author: witgo Author: GuoQiang Li Closes #1341 from witgo/history_env and squashes the following commits: b4fd9f8 [GuoQiang Li] review commit 0ebe401 [witgo] *-history-server.sh load spark-config.sh (cherry picked from commit 9f04db17e50568d5580091add9100693177d7c4f) Signed-off-by: Andrew Or --- sbin/start-history-server.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index e30493da32a7a..580ab471b8a79 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -27,6 +27,9 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` +. "$sbin/spark-config.sh" +. "$SPARK_PREFIX/bin/load-spark-env.sh" + if [ $# != 0 ]; then echo "Using command line arguments for setting the log directory is deprecated. Please " echo "set the spark.history.fs.logDirectory configuration option instead." From 48a07490fdd0e79a34e66e5c1baad0b1558bbda5 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 25 Aug 2014 22:56:35 -0700 Subject: [PATCH 102/192] [Spark-3222] [SQL] Cross join support in HiveQL We can simple treat cross join as inner join without join conditions. Author: Daoyuan Wang Author: adrian-wang Closes #2124 from adrian-wang/crossjoin and squashes the following commits: 8c9b7c5 [Daoyuan Wang] add a test 7d47bbb [adrian-wang] add cross join support for hql (cherry picked from commit 52fbdc2deddcdba02bf5945a36e15870021ec890) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala | 1 + sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 1 + .../golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 | 0 .../golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 | 0 .../golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 | 0 5 files changed, 2 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 create mode 100644 sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 create mode 100644 sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 210753efe7678..66243879b9019 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -310,6 +310,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_nested_type", "create_skewed_table1", "create_struct_table", + "cross_join", "ct_case_insensitive", "database_location", "database_properties", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5da6e8df03aee..581332e600183 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -772,6 +772,7 @@ private[hive] object HiveQl { val joinType = joinToken match { case "TOK_JOIN" => Inner + case "TOK_CROSSJOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter case "TOK_LEFTOUTERJOIN" => LeftOuter case "TOK_FULLOUTERJOIN" => FullOuter diff --git a/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 b/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 b/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 b/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 new file mode 100644 index 0000000000000..e69de29bb2d1d From 0f947f1239831a6ed3b47af65816715999bbe57b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 25 Aug 2014 23:36:09 -0700 Subject: [PATCH 103/192] [SPARK-2886] Use more specific actor system name than "spark" As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001. ``` 14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001. ``` This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality. Author: Andrew Or Closes #1810 from andrewor14/service-name and squashes the following commits: 8c459ed [Andrew Or] Use a common variable for driver/executor actor system names 3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor 921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name c8c6a62 [Andrew Or] Do not include hyphens in actor name 1c1b42e [Andrew Or] Avoid spaces in akka system name f644b55 [Andrew Or] Use more specific service name (cherry picked from commit b21ae5bbb9baa966f69303a30659aa8bbb2098da) Signed-off-by: Andrew Or --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 9 ++++++--- .../spark/scheduler/cluster/SimrSchedulerBackend.scala | 8 +++++--- .../cluster/SparkDeploySchedulerBackend.scala | 8 +++++--- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 5 +++-- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++-- .../streaming/receiver/ReceiverSupervisorImpl.scala | 10 +++++----- .../apache/spark/deploy/yarn/ExecutorLauncher.scala | 9 ++++++--- .../spark/deploy/yarn/YarnAllocationHandler.scala | 8 +++++--- .../apache/spark/deploy/yarn/ExecutorLauncher.scala | 9 ++++++--- .../spark/deploy/yarn/YarnAllocationHandler.scala | 5 +++-- 10 files changed, 47 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index fc36e37c53f5e..72716567ca99b 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -111,6 +111,9 @@ object SparkEnv extends Logging { private val env = new ThreadLocal[SparkEnv] @volatile private var lastSetSparkEnv : SparkEnv = _ + private[spark] val driverActorSystemName = "sparkDriver" + private[spark] val executorActorSystemName = "sparkExecutor" + def set(e: SparkEnv) { lastSetSparkEnv = e env.set(e) @@ -146,9 +149,9 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf) - - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, - securityManager = securityManager) + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + actorSystemName, hostname, port, conf, securityManager) // Figure out which port Akka actually bound to in case the original port is 0 or occupied. // This is so that we tell the executors the correct port to connect to. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d99c76117c168..4f7133c4bc17c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.scheduler.TaskSchedulerImpl private[spark] class SimrSchedulerBackend( @@ -38,8 +38,10 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sc.conf.get("spark.driver.host"), + sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 589dba2e40d20..32138e5246700 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -42,8 +42,10 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 9f45400bcf852..f0172504c55aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -130,7 +130,8 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index d6afb73b74242..e2d32c859bbda 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -27,7 +27,7 @@ import akka.pattern.ask import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} /** * Various utility classes for working with Akka. @@ -192,10 +192,11 @@ private[spark] object AkkaUtils extends Logging { } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { + val driverActorSystemName = SparkEnv.driverActorSystemName val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" + val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index d934b9cbfc3e8..53a3e6200e340 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -20,22 +20,21 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import akka.actor.{Actor, Props} import akka.pattern.ask +import com.google.common.base.Throwables + import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler.DeregisterReceiver import org.apache.spark.streaming.scheduler.AddBlock -import scala.Some import org.apache.spark.streaming.scheduler.RegisterReceiver -import com.google.common.base.Throwables /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -56,7 +55,8 @@ private[streaming] class ReceiverSupervisorImpl( private val trackerActor = { val ip = env.conf.get("spark.driver.host", "localhost") val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format( + SparkEnv.driverActorSystemName, ip, port) env.actorSystem.actorSelection(url) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index c3310fbc24a98..155dd88aa2b81 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -210,8 +210,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 80e0162e9f277..568a6ef932bbd 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -245,8 +245,10 @@ private[yarn] class YarnAllocationHandler( // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + executorHostname) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 45925f1fea005..e093fe4ae6ff8 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -174,8 +174,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 29ccec2adcac3..0a461749c819d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -262,7 +262,8 @@ private[yarn] class YarnAllocationHandler( numExecutorsRunning.decrementAndGet() } else { val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) From 3a9d874d7a46ab8b015631d91ba479d9a0ba827f Mon Sep 17 00:00:00 2001 From: chutium Date: Tue, 26 Aug 2014 11:51:26 -0700 Subject: [PATCH 104/192] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext There are 4 different compression codec available for ```ParquetOutputFormat``` in Spark SQL, it was set as a hard-coded value in ```ParquetRelation.defaultCompression``` original discuss: https://github.com/apache/spark/pull/195#discussion-diff-11002083 i added a new config property in SQLConf to allow user to change this compression codec, and i used similar short names syntax as described in SPARK-2953 #1873 (https://github.com/apache/spark/pull/1873/files#diff-0) btw, which codec should we use as default? it was set to GZIP (https://github.com/apache/spark/pull/195/files#diff-4), but i think maybe we should change this to SNAPPY, since SNAPPY is already the default codec for shuffling in spark-core (SPARK-2469, #1415), and parquet-mr supports Snappy codec natively (https://github.com/Parquet/parquet-mr/commit/e440108de57199c12d66801ca93804086e7f7632). Author: chutium Closes #2039 from chutium/parquet-compression and squashes the following commits: 2f44964 [chutium] [SPARK-3131][SQL] parquet compression default codec set to snappy, also in test suite e578e21 [chutium] [SPARK-3131][SQL] compression codec config property name and default codec set to snappy 21235dc [chutium] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext (cherry picked from commit 8856c3d86009295be871989a5dc7270f31b420cd) Signed-off-by: Michael Armbrust --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../spark/sql/parquet/ParquetRelation.scala | 14 ++- .../spark/sql/parquet/ParquetQuerySuite.scala | 94 +++++++++++++++++++ 3 files changed, 107 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 5cc41a83cc792..f0df19112ae37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -33,6 +33,7 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" + val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -78,6 +79,9 @@ trait SQLConf { /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + /** The compression codec for writing to a Parquetfile */ + private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 1713ae6fb5d93..5ae768293a22e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -100,8 +100,13 @@ private[sql] object ParquetRelation { // The compression type type CompressionType = parquet.hadoop.metadata.CompressionCodecName - // The default compression - val defaultCompression = CompressionCodecName.GZIP + // The parquet compression short names + val shortParquetCompressionCodecNames = Map( + "NONE" -> CompressionCodecName.UNCOMPRESSED, + "UNCOMPRESSED" -> CompressionCodecName.UNCOMPRESSED, + "SNAPPY" -> CompressionCodecName.SNAPPY, + "GZIP" -> CompressionCodecName.GZIP, + "LZO" -> CompressionCodecName.LZO) /** * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that @@ -141,9 +146,8 @@ private[sql] object ParquetRelation { conf: Configuration, sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) - if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { - conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) - } + conf.set(ParquetOutputFormat.COMPRESSION, shortParquetCompressionCodecNames.getOrElse( + sqlContext.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 172dcd6aa0ee3..28f43b36832ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -186,6 +186,100 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) } + test("Compression options for writing to a Parquetfile") { + val defaultParquetCompressionCodec = TestSQLContext.parquetCompressionCodec + import scala.collection.JavaConversions._ + + val file = getTempFilePath("parquet") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec + rdd.saveAsParquetFile(path) + var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "UNCOMPRESSED" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "UNCOMPRESSED") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "none" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "none") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === "UNCOMPRESSED" :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test gzip compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "gzip") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test snappy compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "snappy") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // TODO: Lzo requires additional external setup steps so leave it out for now + // ref.: https://github.com/Parquet/parquet-mr/blob/parquet-1.5.0/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java#L169 + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, defaultParquetCompressionCodec) + } + test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) From 83d273023b03faa0ceacd69956a132f40d247bc1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 26 Aug 2014 13:04:30 -0700 Subject: [PATCH 105/192] [SPARK-2871] [PySpark] add histgram() API RDD.histogram(buckets) Compute a histogram using the provided buckets. The buckets are all open to the right except for the last which is closed. e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 and 50 we would have a histogram of 1,0,1. If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), this can be switched from an O(log n) inseration to O(1) per element(where n = # buckets). Buckets must be sorted and not contain any duplicates, must be at least two elements. If `buckets` is a number, it will generates buckets which is evenly spaced between the minimum and maximum of the RDD. For example, if the min value is 0 and the max is 100, given buckets as 2, the resulting buckets will be [0,50) [50,100]. buckets must be at least 1 If the RDD contains infinity, NaN throws an exception If the elements in RDD do not vary (max == min) always returns a single bucket. It will return an tuple of buckets and histogram. >>> rdd = sc.parallelize(range(51)) >>> rdd.histogram(2) ([0, 25, 50], [25, 26]) >>> rdd.histogram([0, 5, 25, 50]) ([0, 5, 25, 50], [5, 20, 26]) >>> rdd.histogram([0, 15, 30, 45, 60], True) ([0, 15, 30, 45, 60], [15, 15, 15, 6]) >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) >>> rdd.histogram(("a", "b", "c")) (('a', 'b', 'c'), [2, 2]) closes #122, it's duplicated. Author: Davies Liu Closes #2091 from davies/histgram and squashes the following commits: a322f8a [Davies Liu] fix deprecation of e.message 84e85fa [Davies Liu] remove evenBuckets, add more tests (including str) d9a0722 [Davies Liu] address comments 0e18a2d [Davies Liu] add histgram() API (cherry picked from commit 3cedc4f4d78e093fd362085e0a077bb9e4f28ca5) Signed-off-by: Josh Rosen --- python/pyspark/rdd.py | 129 +++++++++++++++++++++++++++++++++++++++- python/pyspark/tests.py | 104 ++++++++++++++++++++++++++++++++ 2 files changed, 232 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f0706d846d014..bfefc261233a5 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -32,7 +32,7 @@ import heapq import bisect from random import Random -from math import sqrt, log +from math import sqrt, log, isinf, isnan from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ @@ -856,6 +856,133 @@ def redFunc(left_counter, right_counter): return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc) + def histogram(self, buckets): + """ + Compute a histogram using the provided buckets. The buckets + are all open to the right except for the last which is closed. + e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], + which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 + and 50 we would have a histogram of 1,0,1. + + If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), + this can be switched from an O(log n) inseration to O(1) per + element(where n = # buckets). + + Buckets must be sorted and not contain any duplicates, must be + at least two elements. + + If `buckets` is a number, it will generates buckets which are + evenly spaced between the minimum and maximum of the RDD. For + example, if the min value is 0 and the max is 100, given buckets + as 2, the resulting buckets will be [0,50) [50,100]. buckets must + be at least 1 If the RDD contains infinity, NaN throws an exception + If the elements in RDD do not vary (max == min) always returns + a single bucket. + + It will return an tuple of buckets and histogram. + + >>> rdd = sc.parallelize(range(51)) + >>> rdd.histogram(2) + ([0, 25, 50], [25, 26]) + >>> rdd.histogram([0, 5, 25, 50]) + ([0, 5, 25, 50], [5, 20, 26]) + >>> rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets + ([0, 15, 30, 45, 60], [15, 15, 15, 6]) + >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) + >>> rdd.histogram(("a", "b", "c")) + (('a', 'b', 'c'), [2, 2]) + """ + + if isinstance(buckets, (int, long)): + if buckets < 1: + raise ValueError("number of buckets must be >= 1") + + # filter out non-comparable elements + def comparable(x): + if x is None: + return False + if type(x) is float and isnan(x): + return False + return True + + filtered = self.filter(comparable) + + # faster than stats() + def minmax(a, b): + return min(a[0], b[0]), max(a[1], b[1]) + try: + minv, maxv = filtered.map(lambda x: (x, x)).reduce(minmax) + except TypeError as e: + if " empty " in str(e): + raise ValueError("can not generate buckets from empty RDD") + raise + + if minv == maxv or buckets == 1: + return [minv, maxv], [filtered.count()] + + try: + inc = (maxv - minv) / buckets + except TypeError: + raise TypeError("Can not generate buckets with non-number in RDD") + + if isinf(inc): + raise ValueError("Can not generate buckets with infinite value") + + # keep them as integer if possible + if inc * buckets != maxv - minv: + inc = (maxv - minv) * 1.0 / buckets + + buckets = [i * inc + minv for i in range(buckets)] + buckets.append(maxv) # fix accumulated error + even = True + + elif isinstance(buckets, (list, tuple)): + if len(buckets) < 2: + raise ValueError("buckets should have more than one value") + + if any(i is None or isinstance(i, float) and isnan(i) for i in buckets): + raise ValueError("can not have None or NaN in buckets") + + if sorted(buckets) != list(buckets): + raise ValueError("buckets should be sorted") + + if len(set(buckets)) != len(buckets): + raise ValueError("buckets should not contain duplicated values") + + minv = buckets[0] + maxv = buckets[-1] + even = False + inc = None + try: + steps = [buckets[i + 1] - buckets[i] for i in range(len(buckets) - 1)] + except TypeError: + pass # objects in buckets do not support '-' + else: + if max(steps) - min(steps) < 1e-10: # handle precision errors + even = True + inc = (maxv - minv) / (len(buckets) - 1) + + else: + raise TypeError("buckets should be a list or tuple or number(int or long)") + + def histogram(iterator): + counters = [0] * len(buckets) + for i in iterator: + if i is None or (type(i) is float and isnan(i)) or i > maxv or i < minv: + continue + t = (int((i - minv) / inc) if even + else bisect.bisect_right(buckets, i) - 1) + counters[t] += 1 + # add last two together + last = counters.pop() + counters[-1] += last + return [counters] + + def mergeCounters(a, b): + return [i + j for i, j in zip(a, b)] + + return buckets, self.mapPartitions(histogram).reduce(mergeCounters) + def mean(self): """ Compute the mean of this RDD's elements. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 51bfbb47e53c2..1db922f513743 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -364,6 +364,110 @@ def test_zip_with_different_number_of_items(self): self.assertEquals(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) + def test_histogram(self): + # empty + rdd = self.sc.parallelize([]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertRaises(ValueError, lambda: rdd.histogram(1)) + + # out of range + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + + # in range with one bucket + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals([4], rdd.histogram([0, 10])[1]) + self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + + # in range with one bucket exact match + self.assertEquals([4], rdd.histogram([1, 4])[1]) + + # out of range with two buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + + # out of range with two uneven buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + + # in range with two buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two bucket and None + rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two uneven buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + + # mixed range with two uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) + self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + + # mixed range with four uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # mixed range with uneven buckets and NaN + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, + 199.0, 200.0, 200.1, None, float('nan')]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # out of range with infinite buckets + rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) + self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + + # invalid buckets + self.assertRaises(ValueError, lambda: rdd.histogram([])) + self.assertRaises(ValueError, lambda: rdd.histogram([1])) + self.assertRaises(ValueError, lambda: rdd.histogram(0)) + self.assertRaises(TypeError, lambda: rdd.histogram({})) + + # without buckets + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + + # without buckets single element + rdd = self.sc.parallelize([1]) + self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + + # without bucket no range + rdd = self.sc.parallelize([1] * 4) + self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + + # without buckets basic two + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + + # without buckets with more requested than elements + rdd = self.sc.parallelize([1, 2]) + buckets = [1 + 0.2 * i for i in range(6)] + hist = [1, 0, 0, 0, 1] + self.assertEquals((buckets, hist), rdd.histogram(5)) + + # invalid RDDs + rdd = self.sc.parallelize([1, float('inf')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + rdd = self.sc.parallelize([float('nan')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + + # string + rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) + self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + + # mixed RDD + rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) + self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + class TestIO(PySparkTestCase): From 35a5853551fe0cd6c51eafb3052f52345d3dcb3c Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 13:22:55 -0700 Subject: [PATCH 106/192] [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull. Make `ScalaReflection` be able to handle like: - `Seq[Int]` as `ArrayType(IntegerType, containsNull = false)` - `Seq[java.lang.Integer]` as `ArrayType(IntegerType, containsNull = true)` - `Map[Int, Long]` as `MapType(IntegerType, LongType, valueContainsNull = false)` - `Map[Int, java.lang.Long]` as `MapType(IntegerType, LongType, valueContainsNull = true)` Author: Takuya UESHIN Closes #1889 from ueshin/issues/SPARK-2969 and squashes the following commits: 24f1c5c [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Python API. 79f5b65 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Java API. 7cd1a7a [Takuya UESHIN] Fix json test failures. 2cfb862 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true. 2f38e61 [Takuya UESHIN] Revert the default value of MapTypes.valueContainsNull. 9fa02f5 [Takuya UESHIN] Fix a test failure. 1a9a96b [Takuya UESHIN] Modify ScalaReflection to handle ArrayType.containsNull and MapType.valueContainsNull. (cherry picked from commit 98c2bb0bbde6fb2b6f64af3efffefcb0dae94c12) Signed-off-by: Michael Armbrust --- python/pyspark/sql.py | 6 ++-- .../spark/sql/catalyst/ScalaReflection.scala | 9 ++++-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +-- .../sql/catalyst/ScalaReflectionSuite.scala | 22 +++++++++++-- .../apache/spark/sql/api/java/DataType.java | 4 +-- .../org/apache/spark/sql/DataTypeSuite.scala | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 32 +++++++++---------- 7 files changed, 49 insertions(+), 30 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d4ca0cc8f336e..0ff6a548a85f1 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -186,15 +186,15 @@ class ArrayType(DataType): """ - def __init__(self, elementType, containsNull=False): + def __init__(self, elementType, containsNull=True): """Creates an ArrayType :param elementType: the data type of elements. :param containsNull: indicates whether the list contains None values. - >>> ArrayType(StringType) == ArrayType(StringType, False) + >>> ArrayType(StringType) == ArrayType(StringType, True) True - >>> ArrayType(StringType, True) == ArrayType(StringType) + >>> ArrayType(StringType, False) == ArrayType(StringType) False """ self.elementType = elementType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 0d26b52a84695..6b6b636cd96dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -62,11 +62,14 @@ object ScalaReflection { sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t - Schema(ArrayType(schemaFor(elementType).dataType), nullable = true) + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< typeOf[Map[_,_]] => val TypeRef(_, _, Seq(keyType, valueType)) = t - Schema(MapType(schemaFor(keyType).dataType, schemaFor(valueType).dataType), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index b52ee6d3378a3..70c6d06cf2534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -270,8 +270,8 @@ case object FloatType extends FractionalType { } object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index e75373d5a74a7..428607d8c8253 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -57,7 +57,9 @@ case class OptionalData( case class ComplexData( arrayField: Seq[Int], - mapField: Map[Int, String], + arrayFieldContainsNull: Seq[java.lang.Integer], + mapField: Map[Int, Long], + mapFieldValueContainsNull: Map[Int, java.lang.Long], structField: PrimitiveData) case class GenericData[A]( @@ -116,8 +118,22 @@ class ScalaReflectionSuite extends FunSuite { val schema = schemaFor[ComplexData] assert(schema === Schema( StructType(Seq( - StructField("arrayField", ArrayType(IntegerType), nullable = true), - StructField("mapField", MapType(IntegerType, StringType), nullable = true), + StructField( + "arrayField", + ArrayType(IntegerType, containsNull = false), + nullable = true), + StructField( + "arrayFieldContainsNull", + ArrayType(IntegerType, containsNull = true), + nullable = true), + StructField( + "mapField", + MapType(IntegerType, LongType, valueContainsNull = false), + nullable = true), + StructField( + "mapFieldValueContainsNull", + MapType(IntegerType, LongType, valueContainsNull = true), + nullable = true), StructField( "structField", StructType(Seq( diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 3eccddef88134..37b4c8ffcba0b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -86,14 +86,14 @@ public abstract class DataType { /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). - * The field of {@code containsNull} is set to {@code false}. + * The field of {@code containsNull} is set to {@code true}. */ public static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, false); + return new ArrayType(elementType, true); } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index cf7d79f42db1d..8fb59c5830f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -24,7 +24,7 @@ class DataTypeSuite extends FunSuite { test("construct an ArrayType") { val array = ArrayType(StringType) - assert(ArrayType(StringType, false) === array) + assert(ArrayType(StringType, true) === array) } test("construct an MapType") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 58b1e23891a3b..05513a127150c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -130,11 +130,11 @@ class JsonSuite extends QueryTest { checkDataType( ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, true)) checkDataType( ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) + ArrayType(IntegerType, false), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -201,26 +201,26 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldAndType) val expectedSchema = StructType( - StructField("arrayOfArray1", ArrayType(ArrayType(StringType)), true) :: - StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true) :: - StructField("arrayOfBigInteger", ArrayType(DecimalType), true) :: - StructField("arrayOfBoolean", ArrayType(BooleanType), true) :: - StructField("arrayOfDouble", ArrayType(DoubleType), true) :: - StructField("arrayOfInteger", ArrayType(IntegerType), true) :: - StructField("arrayOfLong", ArrayType(LongType), true) :: + StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType, false), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: + StructField("arrayOfLong", ArrayType(LongType, false), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: - StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfString", ArrayType(StringType, false), true) :: StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: StructField("field2", StringType, true) :: - StructField("field3", StringType, true) :: Nil)), true) :: + StructField("field3", StringType, true) :: Nil), false), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: StructField("field2", DecimalType, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( - StructField("field1", ArrayType(IntegerType), true) :: - StructField("field2", ArrayType(StringType), true) :: Nil), true) :: Nil) + StructField("field1", ArrayType(IntegerType, false), true) :: + StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -441,7 +441,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(IntegerType), true) :: + StructField("array", ArrayType(IntegerType, false), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -467,7 +467,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true) :: Nil) + StructField("field", LongType, true) :: Nil), false), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -492,7 +492,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(IntegerType), true) :: + StructField("c", ArrayType(IntegerType, false), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: Nil) From 5d981a49c846db8e08bba08d46cf4bba45609e76 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 15:04:08 -0700 Subject: [PATCH 107/192] [SPARK-3063][SQL] ExistingRdd should convert Map to catalyst Map. Currently `ExistingRdd.convertToCatalyst` doesn't convert `Map` value. Author: Takuya UESHIN Closes #1963 from ueshin/issues/SPARK-3063 and squashes the following commits: 3ba41f2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 4d7bae2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 9321379 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 d8a900a [Takuya UESHIN] Make ExistingRdd.convertToCatalyst be able to convert Map value. (cherry picked from commit 6b5584ef1c605cd30f25dbe7099ab32aea1746fb) Signed-off-by: Michael Armbrust --- .../spark/sql/execution/basicOperators.scala | 3 +- .../sql/ScalaReflectionRelationSuite.scala | 46 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f9dfa3c92f1eb..374af48b820c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -206,7 +206,8 @@ case class Sort( object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case o: Option[_] => o.orNull - case s: Seq[Any] => s.map(convertToCatalyst) + case s: Seq[_] => s.map(convertToCatalyst) + case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 5b84c658db942..e24c521d24c7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -21,6 +21,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext._ case class ReflectData( @@ -56,6 +57,22 @@ case class OptionalReflectData( case class ReflectBinary(data: Array[Byte]) +case class Nested(i: Option[Int], s: String) + +case class Data( + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapContainsNul: Map[Int, Option[Long]], + nested: Nested) + +case class ComplexReflectData( + arrayField: Seq[Int], + arrayFieldContainsNull: Seq[Option[Int]], + mapField: Map[Int, Long], + mapFieldContainsNull: Map[Int, Option[Long]], + dataField: Data) + class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -90,4 +107,33 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } + + test("query complex data") { + val data = ComplexReflectData( + Seq(1, 2, 3), + Seq(Some(1), Some(2), None), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), + Data( + Seq(10, 20, 30), + Seq(Some(10), Some(20), None), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), + Nested(None, "abc"))) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerTempTable("reflectComplexData") + + assert(sql("SELECT * FROM reflectComplexData").collect().head === + new GenericRow(Array[Any]( + Seq(1, 2, 3), + Seq(1, 2, null), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> 10L, 2 -> 20L, 3 -> null), + new GenericRow(Array[Any]( + Seq(10, 20, 30), + Seq(10, 20, null), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> 100L, 20 -> 200L, 30 -> null), + new GenericRow(Array[Any](null, "abc"))))))) + } } From 5ff90008603f166d63555a7c617f9409b2d183a4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 26 Aug 2014 15:12:27 -0700 Subject: [PATCH 108/192] [SPARK-3226][MLLIB] doc update for native libraries to mention `-Pnetlib-lgpl` option. atalwalkar Author: Xiangrui Meng Closes #2128 from mengxr/mllib-native and squashes the following commits: 4cbba57 [Xiangrui Meng] update mllib dependencies (cherry picked from commit adbd5c1636669fc474ab02b54cd1ced353f68712) Signed-off-by: Xiangrui Meng --- docs/mllib-guide.md | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index ca0a84a8c53fd..4d4198b9e0452 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,18 +35,23 @@ and the migration guide below will explain all changes between releases. # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java), and -[jblas](https://github.com/mikiobraun/jblas). +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), +which depends on [netlib-java](https://github.com/fommil/netlib-java), +and [jblas](https://github.com/mikiobraun/jblas). `netlib-java` and `jblas` depend on native Fortran routines. You need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not -already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries -automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's -dependency set. If no native library is available at runtime, you will see a warning message. To -use native libraries from `netlib-java`, please include artifact -`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see -[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) +if it is not already present on your nodes. +MLlib will throw a linking error if it cannot detect these libraries automatically. +Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set under default settings. +If no native library is available at runtime, you will see a warning message. +To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or +include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. +If you want to use optimized BLAS/LAPACK libraries such as +[OpenBLAS](http://www.openblas.net/), please link its shared libraries to +`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively. +BLAS/LAPACK libraries on worker nodes should be built without multithreading. To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. From 2715eb77b8613d3df30fc718ab382d32c0635d47 Mon Sep 17 00:00:00 2001 From: Burak Date: Tue, 26 Aug 2014 15:18:42 -0700 Subject: [PATCH 109/192] [SPARK-2839][MLlib] Stats Toolkit documentation updated Documentation updated for the Statistics Toolkit of MLlib. mengxr atalwalkar https://issues.apache.org/jira/browse/SPARK-2839 P.S. Accidentally closed #2123. New commits didn't show up after I reopened the PR. I've opened this instead and closed the old one. Author: Burak Closes #2130 from brkyvz/StatsLib-Docs and squashes the following commits: a54a855 [Burak] [SPARK-2839][MLlib] Addressed comments bfc6896 [Burak] [SPARK-2839][MLlib] Added a more specific link to colStats() for pyspark 213fe3f [Burak] [SPARK-2839][MLlib] Modifications made according to review fec4d9d [Burak] [SPARK-2830][MLlib] Stats Toolkit documentation updated (cherry picked from commit 1208f72ac78960fe5060187761479b2a9a417c1b) Signed-off-by: Xiangrui Meng --- docs/mllib-stats.md | 372 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 331 insertions(+), 41 deletions(-) diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index f25dca746ba3a..511a9fbf710cc 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,6 +25,85 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` +## Summary Statistics + +We provide column summary statistics for `RDD[Vector]` through the function `colStats` +available in `Statistics`. + +
    +
    + +[`colStats()`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} + +val observations: RDD[Vector] = ... // an RDD of Vectors + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzeros) // number of nonzeros in each column + +{% endhighlight %} +
    + +
    + +[`colStats()`](api/java/org/apache/spark/mllib/stat/Statistics.html) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaRDD mat = ... // an RDD of Vectors + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +{% endhighlight %} +
    + +
    +[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +mat = ... # an RDD of Vectors + +# Compute column summary statistics. +summary = Statistics.colStats(mat) +print summary.mean() +print summary.variance() +print summary.numNonzeros() + +{% endhighlight %} +
    + +
    + ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. @@ -99,69 +178,280 @@ v = u.map(lambda x: 1.0 + 2.0 * x)
    -## Stratified Sampling +## Correlations calculation -## Summary Statistics +Calculating the correlation between two series of data is a common operation in Statistics. In MLlib +we provide the flexibility to calculate pairwise correlations among many series. The supported +correlation methods are currently Pearson's and Spearman's correlation. + +
    +
    +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.stat.Statistics + +val sc: SparkContext = ... + +val seriesX: RDD[Double] = ... // a series +val seriesY: RDD[Double] = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") + +val data: RDD[Vector] = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +val correlMatrix: Matrix = Statistics.corr(data, "pearson") + +{% endhighlight %} +
    + +
    +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or +a `JavaRDD`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight java %} +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaDoubleRDD seriesX = ... // a series +JavaDoubleRDD seriesY = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + +JavaRDD data = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); + +{% endhighlight %} +
    -### Multivariate summary statistics +
    +[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +seriesX = ... # a series +seriesY = ... # must have the same number of partitions and cardinality as seriesX -We provide column summary statistics for `RowMatrix` (note: this functionality is not currently supported in `IndexedRowMatrix` or `CoordinateMatrix`). -If the number of columns is not large, e.g., on the order of thousands, then the -covariance matrix can also be computed as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the -number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, -and is faster if the rows are sparse. +# Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +# method is not specified, Pearson's method will be used by default. +print Statistics.corr(seriesX, seriesY, method="pearson") + +data = ... # an RDD of Vectors +# calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +# If a method is not specified, Pearson's method will be used by default. +print Statistics.corr(data, method="pearson") + +{% endhighlight %} +
    + +
    + +## Stratified sampling + +Unlike the other statistics functions, which reside in MLLib, stratified sampling methods, +`sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified +sampling, the keys can be thought of as a label and the value as a specific attribute. For example +the key can be man or woman, or document ids, and the respective values can be the list of ages +of the people in the population or the list of words in the documents. The `sampleByKey` method +will flip a coin to decide whether an observation will be sampled or not, therefore requires one +pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant +more resources than the per-stratum simple random sampling used in `sampleByKey`, but will provide +the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in +python.
    - -[`computeColumnSummaryStatistics()`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of -[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.PairRDDFunctions -val mat: RowMatrix = ... // a RowMatrix +val sc: SparkContext = ... -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column +val data = ... // an RDD[(K, V)] of any key value pairs +val fractions: Map[K, Double] = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +val approxSample = data.sampleByKey(withReplacement = false, fractions) +val exactSample = data.sampleByKeyExact(withReplacement = false, fractions) -// Compute the covariance matrix. -val cov: Matrix = mat.computeCovariance() {% endhighlight %}
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of -[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/java/org/apache/spark/api/java/JavaPairRDD.html) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import java.util.Map; -RowMatrix mat = ... // a RowMatrix +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; -// Compute column summary statistics. -MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column +JavaSparkContext jsc = ... + +JavaPairRDD data = ... // an RDD of any key value pairs +Map fractions = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +JavaPairRDD approxSample = data.sampleByKey(false, fractions); +JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); + +{% endhighlight %} +
    +
    +[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the +desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the +set of keys. + +*Note:* `sampleByKeyExact()` is currently not supported in Python. + +{% highlight python %} + +sc = ... # SparkContext + +data = ... # an RDD of any key value pairs +fractions = ... # specify the exact fraction desired from each key as a dictionary + +approxSample = data.sampleByKey(False, fractions); -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); {% endhighlight %}
    + +
    + +## Hypothesis testing + +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. MLlib currently supports Pearson's +chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine +whether the goodness of fit or the independence test is conducted. The goodness of fit test requires +an input type of `Vector`, whereas the independence test requires a `Matrix` as input. + +MLlib also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared +independence tests. + +
    +
    +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics._ + +val sc: SparkContext = ... + +val vec: Vector = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +val goodnessOfFitTestResult = Statistics.chiSqTest(vec) +println(goodnessOfFitTestResult) // summary of the test including the p-value, degrees of freedom, + // test statistic, the method used, and the null hypothesis. + +val mat: Matrix = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +val independenceTestResult = Statistics.chiSqTest(mat) +println(independenceTestResult) // summary of the test including the p-value, degrees of freedom... + +val obs: RDD[LabeledPoint] = ... // (feature, label) pairs. + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) +var i = 1 +featureTestResults.foreach { result => + println(s"Column $i:\n$result") + i += 1 +} // summary of the test + +{% endhighlight %}
    +
    +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.ChiSqTestResult; + +JavaSparkContext jsc = ... -## Hypothesis Testing +Vector vec = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); +// summary of the test including the p-value, degrees of freedom, test statistic, the method used, +// and the null hypothesis. +System.out.println(goodnessOfFitTestResult); + +Matrix mat = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); +// summary of the test including the p-value, degrees of freedom... +System.out.println(independenceTestResult); + +JavaRDD obs = ... // an RDD of labeled points + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); +int i = 1; +for (ChiSqTestResult result : featureTestResults) { + System.out.println("Column " + i + ":"); + System.out.println(result); // summary of the test + i++; +} + +{% endhighlight %} +
    + +
    From a308a1624fd5783422149e641aa748fec9bf4c63 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 26 Aug 2014 16:29:14 -0700 Subject: [PATCH 110/192] [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences It is common to want to describe sets of attributes that are in various parts of a query plan. However, the semantics of putting `AttributeReference` objects into a standard Scala `Set` result in subtle bugs when references differ cosmetically. For example, with case insensitive resolution it is possible to have two references to the same attribute whose names are not equal. In this PR I introduce a new abstraction, an `AttributeSet`, which performs all comparisons using the globally unique `ExpressionId` instead of case class equality. (There is already a related class, [`AttributeMap`](https://github.com/marmbrus/spark/blob/inMemStats/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala#L32)) This new type of set is used to fix a bug in the optimizer where needed attributes were getting projected away underneath join operators. I also took this opportunity to refactor the expression and query plan base classes. In all but one instance the logic for computing the `references` of an `Expression` were the same. Thus, I moved this logic into the base class. For query plans the semantics of the `references` method were ill defined (is it the references output? or is it those used by expression evaluation? or what?). As a result, this method wasn't really used very much. So, I removed it. TODO: - [x] Finish scala doc for `AttributeSet` - [x] Scan the code for other instances of `Set[Attribute]` and refactor them. - [x] Finish removing `references` from `QueryPlan` Author: Michael Armbrust Closes #2109 from marmbrus/attributeSets and squashes the following commits: 1c0dae5 [Michael Armbrust] work on serialization bug. 9ba868d [Michael Armbrust] Merge remote-tracking branch 'origin/master' into attributeSets 3ae5288 [Michael Armbrust] review comments 40ce7f6 [Michael Armbrust] style d577cc7 [Michael Armbrust] Scaladoc cae5d22 [Michael Armbrust] remove more references implementations d6e16be [Michael Armbrust] Remove more instances of "def references" and normal sets of attributes. fc26b49 [Michael Armbrust] Add AttributeSet class, remove references from Expression. (cherry picked from commit c4787a3690a9ed3b8b2c6c294fc4a6915436b6f7) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../sql/catalyst/analysis/unresolved.scala | 1 - .../catalyst/expressions/AttributeSet.scala | 106 ++++++++++++++++++ .../catalyst/expressions/BoundAttribute.scala | 2 - .../sql/catalyst/expressions/Expression.scala | 6 +- .../spark/sql/catalyst/expressions/Rand.scala | 1 - .../sql/catalyst/expressions/ScalaUdf.scala | 1 - .../sql/catalyst/expressions/SortOrder.scala | 1 - .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 25 ++--- .../sql/catalyst/expressions/arithmetic.scala | 2 - .../catalyst/expressions/complexTypes.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 2 - .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 3 - .../sql/catalyst/expressions/predicates.scala | 6 +- .../spark/sql/catalyst/expressions/sets.scala | 5 - .../expressions/stringOperations.scala | 2 - .../sql/catalyst/optimizer/Optimizer.scala | 12 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 11 +- .../plans/logical/ScriptTransformation.scala | 4 +- .../plans/logical/basicOperators.scala | 29 +---- .../catalyst/plans/logical/partitioning.scala | 4 - .../plans/physical/partitioning.scala | 3 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 7 +- .../columnar/InMemoryColumnarTableScan.scala | 2 - .../spark/sql/execution/SparkPlan.scala | 3 +- .../spark/sql/execution/debug/package.scala | 2 - .../spark/sql/execution/pythonUdfs.scala | 2 - .../spark/sql/hive/HiveStrategies.scala | 8 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 5 - .../hive/execution/HiveResolutionSuite.scala | 9 +- 35 files changed, 166 insertions(+), 123 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c18d7858f0a43..4a9524074132e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -132,7 +132,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolveChildren) - val requiredAttributes = resolved.collect { case a: Attribute => a }.toSet + val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { @@ -152,8 +152,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve).toSet - val missingInAggs = resolved -- a.outputSet + val resolved = unresolved.flatMap(groupingRelation.resolve) + val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a0e25775da6dd..a2c61c65487cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -66,7 +66,6 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") - override def references = children.flatMap(_.references).toSet override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala new file mode 100644 index 0000000000000..c3a08bbdb6bc7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -0,0 +1,106 @@ +/* + * 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.sql.catalyst.expressions + +protected class AttributeEquals(val a: Attribute) { + override def hashCode() = a.exprId.hashCode() + override def equals(other: Any) = other match { + case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId + case otherAttribute => false + } +} + +object AttributeSet { + /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ + def apply(baseSet: Seq[Attribute]) = { + new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) + } +} + +/** + * A Set designed to hold [[AttributeReference]] objects, that performs equality checking using + * expression id instead of standard java equality. Using expression id means that these + * sets will correctly test for membership, even when the AttributeReferences in question differ + * cosmetically (e.g., the names have different capitalizations). + * + * Note that we do not override equality for Attribute references as it is really weird when + * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * and also makes doing transformations hard (we always try keep older trees instead of new ones + * when the transformation was a no-op). + */ +class AttributeSet private (val baseSet: Set[AttributeEquals]) + extends Traversable[Attribute] with Serializable { + + /** Returns true if the members of this AttributeSet and other are the same. */ + override def equals(other: Any) = other match { + case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + case _ => false + } + + /** Returns true if this set contains an Attribute with the same expression id as `elem` */ + def contains(elem: NamedExpression): Boolean = + baseSet.contains(new AttributeEquals(elem.toAttribute)) + + /** Returns a new [[AttributeSet]] that contains `elem` in addition to the current elements. */ + def +(elem: Attribute): AttributeSet = // scalastyle:ignore + new AttributeSet(baseSet + new AttributeEquals(elem)) + + /** Returns a new [[AttributeSet]] that does not contain `elem`. */ + def -(elem: Attribute): AttributeSet = + new AttributeSet(baseSet - new AttributeEquals(elem)) + + /** Returns an iterator containing all of the attributes in the set. */ + def iterator: Iterator[Attribute] = baseSet.map(_.a).iterator + + /** + * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in + * `other`. + */ + def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + + /** + * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found + * in `other`. + */ + def --(other: Traversable[NamedExpression]) = + new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) + + /** + * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found + * in `other`. + */ + def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + + /** + * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to + * true. + */ + override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + + /** + * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in + * `this` and `other`. + */ + def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + + override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) + + // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all + // sorts of things in its closure. + override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 0913f15888780..54c6baf1af3bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -32,8 +32,6 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) type EvaluatedType = Any - override def references = Set.empty - override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ba62dabe3dd6a..70507e7ee2be8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -41,7 +41,7 @@ abstract class Expression extends TreeNode[Expression] { */ def foldable: Boolean = false def nullable: Boolean - def references: Set[Attribute] + def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator)) /** Returns the result of evaluating this expression on a given input Row */ def eval(input: Row = null): EvaluatedType @@ -230,8 +230,6 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - override def references = left.references ++ right.references - override def toString = s"($left $symbol $right)" } @@ -242,5 +240,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - override def references = child.references + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 38f836f0a1a0e..851db95b9177e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.types.DoubleType case object Rand extends LeafExpression { override def dataType = DoubleType override def nullable = false - override def references = Set.empty private[this] lazy val rand = new Random diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 95633dd0c9870..63ac2a608b6ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -24,7 +24,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def references = children.flatMap(_.references).toSet def nullable = true /** This method has been generated by this script diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d2b7685e73065..d00b2ac09745c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -31,7 +31,6 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def references = child.references override def dataType = child.dataType override def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index eb8898900d6a5..1eb55715794a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -35,7 +35,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow def nullable = false - def references = children.toSet + def dataType = DynamicType override def eval(input: Row): DynamicRow = input match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 613b87ca98d97..dbc0c2965a805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -78,7 +78,7 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def references = base.references + override def nullable = base.nullable override def dataType = base.dataType @@ -89,7 +89,7 @@ abstract class AggregateFunction } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MIN($child)" @@ -119,7 +119,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MAX($child)" @@ -149,7 +149,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"COUNT($child)" @@ -166,7 +166,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet + override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" @@ -184,7 +184,6 @@ case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpress def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = ArrayType(expressions.head.dataType) override def toString = s"AddToHashSet(${expressions.mkString(",")})" @@ -219,7 +218,6 @@ case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression def this() = this(null) override def children = inputSet :: Nil - override def references = inputSet.references override def nullable = false override def dataType = LongType override def toString = s"CombineAndCount($inputSet)" @@ -248,7 +246,7 @@ case class CombineSetsAndCountFunction( case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -257,7 +255,7 @@ case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -266,7 +264,7 @@ case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -284,7 +282,7 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = DoubleType override def toString = s"AVG($child)" @@ -304,7 +302,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM($child)" @@ -322,7 +320,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" @@ -331,7 +329,6 @@ case class SumDistinct(child: Expression) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references override def nullable = true override def dataType = child.dataType override def toString = s"FIRST($child)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 5f8b6ae10f0c4..aae86a3628be1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -95,8 +95,6 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def children = left :: right :: Nil - override def references = left.references ++ right.references - override def dataType = left.dataType override def eval(input: Row): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index c1154eb81c319..dafd745ec96c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -31,7 +31,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { /** `Null` is returned for invalid ordinals. */ override def nullable = true override def foldable = child.foldable && ordinal.foldable - override def references = children.flatMap(_.references).toSet + def dataType = child.dataType match { case ArrayType(dt, _) => dt case MapType(_, vt, _) => vt diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index e99c5b452d183..9c865254e0be9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -47,8 +47,6 @@ abstract class Generator extends Expression { override def nullable = false - override def references = children.flatMap(_.references).toSet - /** * Should be overridden by specific generators. Called only once for each instance to ensure * that rule application does not change the output schema of a generator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index e15e16d633365..a8c2396d62632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -52,7 +52,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def foldable = true def nullable = value == null - def references = Set.empty + override def toString = if (value != null) value.toString else "null" @@ -66,8 +66,6 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf val dataType = Literal(value).dataType - def references = Set.empty - def update(expression: Expression, input: Row) = { value = expression.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 02d04762629f5..7c4b9d4847e26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -62,7 +62,7 @@ abstract class Attribute extends NamedExpression { def toAttribute = this def newInstance: Attribute - override def references = Set(this) + } /** @@ -85,7 +85,7 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def references = child.references + override def toAttribute = { if (resolved) { @@ -116,6 +116,8 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + override def references = AttributeSet(this :: Nil) + override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index e88c5d4fa178a..086d0a3e073e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -26,7 +26,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ def nullable = !children.exists(!_.nullable) - def references = children.flatMap(_.references).toSet // Coalesce is foldable if all children are foldable. override def foldable = !children.exists(!_.foldable) @@ -53,7 +52,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false @@ -65,7 +63,6 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false override def toString = s"IS NOT NULL $child" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 5976b0ddf3e03..1313ccd120c1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -85,7 +85,7 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def children = value +: list - def references = children.flatMap(_.references).toSet + def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" @@ -197,7 +197,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi def children = predicate :: trueValue :: falseValue :: Nil override def nullable = trueValue.nullable || falseValue.nullable - def references = children.flatMap(_.references).toSet + override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { if (!resolved) { @@ -239,7 +239,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any def children = branches - def references = children.flatMap(_.references).toSet + def dataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index e6c570b47bee2..3d4c4a8853c12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -26,8 +26,6 @@ import org.apache.spark.util.collection.OpenHashSet case class NewSet(elementType: DataType) extends LeafExpression { type EvaluatedType = Any - def references = Set.empty - def nullable = false // We are currently only using these Expressions internally for aggregation. However, if we ever @@ -53,9 +51,6 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { def nullable = set.nullable def dataType = set.dataType - - def references = (item.flatMap(_.references) ++ set.flatMap(_.references)).toSet - def eval(input: Row): Any = { val itemEval = item.eval(input) val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 97fc3a3b14b88..c2a3a5ca3ca8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -226,8 +226,6 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends if (str.dataType == BinaryType) str.dataType else StringType } - def references = children.flatMap(_.references).toSet - override def children = str :: pos :: len :: Nil @inline diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 5f86d6047cb9c..ddd4b3755d629 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -65,8 +65,10 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of all references required either above or to evaluate the condition. - val allReferences: Set[Attribute] = - projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + AttributeSet( + projectList.flatMap(_.references.iterator)) ++ + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) @@ -76,8 +78,8 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from right side of a LeftSemiJoin. case Join(left, right, LeftSemi, condition) => // Collect the list of all references required to evaluate the condition. - val allReferences: Set[Attribute] = - condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) Join(left, prunedChild(right, allReferences), LeftSemi, condition) @@ -104,7 +106,7 @@ object ColumnPruning extends Rule[LogicalPlan] { } /** Applies a projection only when the child is producing unnecessary attributes */ - private def prunedChild(c: LogicalPlan, allReferences: Set[Attribute]) = + private def prunedChild(c: LogicalPlan, allReferences: AttributeSet) = if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 0988b0c6d990c..1e177e28f80b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} @@ -29,7 +29,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** * Returns the set of attributes that are output by this node. */ - def outputSet: Set[Attribute] = output.toSet + def outputSet: AttributeSet = AttributeSet(output) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 278569f0cb14a..8616ac45b0e95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -45,17 +45,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product ) - /** - * Returns the set of attributes that are referenced by this node - * during evaluation. - */ - def references: Set[Attribute] - /** * Returns the set of attributes that this node takes as * input from its children. */ - lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet + lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) /** * Returns true if this expression and all its children have been resolved to a specific schema @@ -126,9 +120,6 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { override lazy val statistics: Statistics = throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") - - // Leaf nodes by definition cannot reference any input attributes. - override def references = Set.empty } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index d3f9d0fb93237..4460c86ed9026 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -30,6 +30,4 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - def references = input.flatMap(_.references).toSet -} + child: LogicalPlan) extends UnaryNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3cb407217c4c3..4adfb189372d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) - def references = projectList.flatMap(_.references).toSet } /** @@ -59,14 +58,10 @@ case class Generate( override def output = if (join) child.output ++ generatorOutput else generatorOutput - - override def references = - if (join) child.outputSet else generator.references } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = condition.references } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -76,8 +71,6 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } - - override def references = Set.empty } case class Join( @@ -86,8 +79,6 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def references = condition.map(_.references).getOrElse(Set.empty) - override def output = { joinType match { case LeftSemi => @@ -106,8 +97,6 @@ case class Join( case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { def output = left.output - - def references = Set.empty } case class InsertIntoTable( @@ -118,7 +107,6 @@ case class InsertIntoTable( extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. override def children = table :: child :: Nil - override def references = Set.empty override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { @@ -130,20 +118,17 @@ case class InsertIntoCreatedTable( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = order.flatMap(_.references).toSet } case class Aggregate( @@ -152,19 +137,20 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + /** The set of all AttributeReferences required for this aggregation. */ + def references = + AttributeSet( + groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) + override def output = aggregateExpressions.map(_.toAttribute) - override def references = - (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = limitExpr.references } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) - override def references = Set.empty } /** @@ -191,20 +177,16 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.qualifiers) case other => other } - - override def references = Set.empty } case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = Set.empty } case class Distinct(child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = child.outputSet } case object NoRelation extends LeafNode { @@ -213,5 +195,4 @@ case object NoRelation extends LeafNode { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override def output = left.output - override def references = Set.empty } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 7146fbd540f29..72b0c5c8e7a26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -31,13 +31,9 @@ abstract class RedistributeData extends UnaryNode { case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) extends RedistributeData { - - def references = sortExpressions.flatMap(_.references).toSet } case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) extends RedistributeData { - - def references = partitionExpressions.flatMap(_.references).toSet } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 4bb022cf238af..ccb0df113c063 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -71,6 +71,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "An AllTuples should be used to represent a distribution that only has " + "a single partition.") + // TODO: This is not really valid... def clustering = ordering.map(_.child).toSet } @@ -139,7 +140,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) with Partitioning { override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType @@ -179,7 +179,6 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) with Partitioning { override def children = ordering - override def references = ordering.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6344874538d67..296202543e2ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq - def references = Set.empty[Attribute] def nullable = true def dataType = NullType override lazy val resolved = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 8a9f4deb6a19e..6f0eed3f63c41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -344,8 +344,8 @@ class SQLContext(@transient val sparkContext: SparkContext) prunePushedDownFilters: Seq[Expression] => Seq[Expression], scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { - val projectSet = projectList.flatMap(_.references).toSet - val filterSet = filterPredicates.flatMap(_.references).toSet + val projectSet = AttributeSet(projectList.flatMap(_.references)) + val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) // Right now we still use a projection even if the only evaluation is applying an alias @@ -354,7 +354,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: Decouple final output schema from expression evaluation so this copy can be // avoided safely. - if (projectList.toSet == projectSet && filterSet.subsetOf(projectSet)) { + if (AttributeSet(projectList.map(_.toAttribute)) == projectSet && + filterSet.subsetOf(projectSet)) { // When it is possible to just use column pruning to get the right projection and // when the columns of this projection are enough to evaluate all filter conditions, // just do a scan followed by a filter, with no extra project. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index e63b4903041f6..24e88eea3189e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -79,8 +79,6 @@ private[sql] case class InMemoryRelation( override def children = Seq.empty - override def references = Set.empty - override def newInstance() = { new InMemoryRelation( output.map(_.newInstance), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 21cbbc9772a00..7d33ea5b021e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -141,10 +141,9 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output - override def references = Set.empty override def children = Nil - override final def newInstance: this.type = { + override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index f31df051824d7..5b896c55b7393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -58,8 +58,6 @@ package object debug { } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { - def references = Set.empty - def output = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index b92091b560b1c..aef6ebf86b1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -49,7 +49,6 @@ private[spark] case class PythonUDF( override def toString = s"PythonUDF#$name(${children.mkString(",")})" def nullable: Boolean = true - def references: Set[Attribute] = children.flatMap(_.references).toSet override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") } @@ -113,7 +112,6 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() - def references = Set.empty def output = child.output :+ resultAttribute } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 389ace726d205..10fa8314c9156 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -79,9 +79,9 @@ private[hive] trait HiveStrategies { hiveContext.convertMetastoreParquet => // Filter out all predicates that only deal with partition keys - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionsKeys = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionsKeys) } // We are going to throw the predicates and projection back at the whole optimization @@ -176,9 +176,9 @@ private[hive] trait HiveStrategies { case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionKeyIds = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionKeyIds) } pruneFilterProject( 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 c6497a15efa0c..7d1ad53d8bdb3 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 @@ -88,7 +88,6 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type EvaluatedType = Any def nullable = true - def references = children.flatMap(_.references).toSet lazy val function = createFunction[UDFType]() @@ -229,8 +228,6 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" def newInstance() = new HiveUdafFunction(functionClassName, children, this) @@ -253,8 +250,6 @@ private[hive] case class HiveGenericUdtf( children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { - override def references = children.flatMap(_.references).toSet - @transient protected lazy val function: GenericUDTF = createFunction() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 6b3ffd1c0ffe2..b6be6bc1bfefe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. @@ -57,6 +57,13 @@ class HiveResolutionSuite extends HiveComparisonTest { .registerTempTable("caseSensitivityTest") sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + + println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) + + sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + + // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + } test("nested repeated resolution") { From c0e1f99f57b540cbcfd880662a778477c70fc94d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 26 Aug 2014 17:33:40 -0700 Subject: [PATCH 111/192] [SPARK-2964] [SQL] Remove duplicated code from spark-sql and start-thriftserver.sh Author: Cheng Lian Author: Kousuke Saruta Closes #1886 from sarutak/SPARK-2964 and squashes the following commits: 8ef8751 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2964 26e7c95 [Kousuke Saruta] Revert "Shorten timeout to more reasonable value" ffb68fa [Kousuke Saruta] Modified spark-sql and start-thriftserver.sh to use bin/utils.sh 8c6f658 [Kousuke Saruta] Merge branch 'spark-3026' of https://github.com/liancheng/spark into SPARK-2964 81b43a8 [Cheng Lian] Shorten timeout to more reasonable value a89e66d [Cheng Lian] Fixed command line options quotation in scripts 9c894d3 [Cheng Lian] Fixed bin/spark-sql -S option typo be4736b [Cheng Lian] Report better error message when running JDBC/CLI without hive-thriftserver profile enabled (cherry picked from commit faeb9c0e1440f4af888be0dfc5de7b57efc92b00) Signed-off-by: Michael Armbrust --- bin/spark-sql | 55 +++++-------------- .../org/apache/spark/deploy/SparkSubmit.scala | 14 ++++- sbin/start-thriftserver.sh | 39 +++++-------- 3 files changed, 39 insertions(+), 69 deletions(-) diff --git a/bin/spark-sql b/bin/spark-sql index 564f1f419060f..2a3cb31f58e8d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,6 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" @@ -43,52 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -CLI_ARGS=() -SUBMISSION_ARGS=() - -while (($#)); do - case $1 in - -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" - -e) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - -s | --silent) - CLI_ARGS+=("$1"); shift - ;; - - -v | --verbose) - # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose - CLI_ARGS+=("$1") - SUBMISSION_ARGS+=("$1"); shift - ;; - - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Spark SQL CLI main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" +exit $exit_status 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 f8cdbc3c392b5..550ee72538900 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,6 +54,8 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] var printStream: PrintStream = System.err @@ -311,8 +313,18 @@ object SparkSubmit { System.setProperty(key, value) } - val mainClass = Class.forName(childMainClass, true, loader) + var mainClass: Class[_] = null + + try { + mainClass = Class.forName(childMainClass, true, loader) + } catch { + case e: ClassNotFoundException => + e.printStackTrace(printStream) + System.exit(CLASS_NOT_FOUND_EXIT_STATUS) + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + try { mainMethod.invoke(null, childArgs.toArray) } catch { diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 2c4452473ccbc..c519a77df4a14 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,6 +27,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +CLASS_NOT_FOUND_EXIT_STATUS=1 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -43,36 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -THRIFT_SERVER_ARGS=() -SUBMISSION_ARGS=() +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" -while (($#)); do - case $1 in - --hiveconf) - ensure_arg_number $# 2 - THRIFT_SERVER_ARGS+=("$1"); shift - THRIFT_SERVER_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Hive Thrift server main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" +exit $exit_status From 0d9723309b56d49c91018dd269ed9873004071b1 Mon Sep 17 00:00:00 2001 From: nchammas Date: Tue, 26 Aug 2014 17:50:04 -0700 Subject: [PATCH 112/192] [Docs] Run tests like in contributing guide The Contributing to Spark guide [recommends](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting) running tests by calling `./dev/run-tests`. The README should, too. `./sbt/sbt test` does not cover Python tests or style tests. Author: nchammas Closes #2149 from nchammas/patch-2 and squashes the following commits: 2b3b132 [nchammas] [Docs] Run tests like in contributing guide (cherry picked from commit 73b3089b8d2901dab11bb1ef6f46c29625b677fe) Signed-off-by: Reynold Xin --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index bec68e82fbeb5..ef4a076842be5 100644 --- a/README.md +++ b/README.md @@ -69,7 +69,7 @@ Many of the example programs print usage help if no params are given. Testing first requires [building Spark](#building-spark). Once Spark is built, tests can be run using: - ./sbt/sbt test + ./dev/run-tests ## A Note About Hadoop Versions From 8b5af6f7494fbe83f5b48e72bcbcb636b96cfe75 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 18:28:41 -0700 Subject: [PATCH 113/192] [SPARK-3036][SPARK-3037][SQL] Add MapType/ArrayType containing null value support to Parquet. JIRA: - https://issues.apache.org/jira/browse/SPARK-3036 - https://issues.apache.org/jira/browse/SPARK-3037 Currently this uses the following Parquet schema for `MapType` when `valueContainsNull` is `true`: ``` message root { optional group a (MAP) { repeated group map (MAP_KEY_VALUE) { required int32 key; optional int32 value; } } } ``` for `ArrayType` when `containsNull` is `true`: ``` message root { optional group a (LIST) { repeated group bag { optional int32 array; } } } ``` We have to think about compatibilities with older version of Spark or Hive or others I mentioned in the JIRA issues. Notice: This PR is based on #1963 and #1889. Please check them first. /cc marmbrus, yhuai Author: Takuya UESHIN Closes #2032 from ueshin/issues/SPARK-3036_3037 and squashes the following commits: 4e8e9e7 [Takuya UESHIN] Add ArrayType containing null value support to Parquet. 013c2ca [Takuya UESHIN] Add MapType containing null value support to Parquet. 62989de [Takuya UESHIN] Merge branch 'issues/SPARK-2969' into issues/SPARK-3036_3037 8e38b53 [Takuya UESHIN] Merge branch 'issues/SPARK-3063' into issues/SPARK-3036_3037 (cherry picked from commit 727cb25bcc29481d6b744abef1ca091e64b5f91f) Signed-off-by: Michael Armbrust --- .../spark/sql/parquet/ParquetConverter.scala | 83 +++++++++++++++++++ .../sql/parquet/ParquetTableSupport.scala | 54 +++++++----- .../spark/sql/parquet/ParquetTypes.scala | 54 ++++++++---- .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +++- 4 files changed, 167 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index ef4526ec03439..9fd6aed402838 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -58,6 +58,7 @@ private[sql] object CatalystConverter { // This is mostly Parquet convention (see, e.g., `ConversionPatterns`). // Note that "array" for the array elements is chosen by ParquetAvro. // Using a different value will result in Parquet silently dropping columns. + val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" @@ -82,6 +83,9 @@ private[sql] object CatalystConverter { case ArrayType(elementType: DataType, false) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } + case ArrayType(elementType: DataType, true) => { + new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) + } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields.toArray, fieldIndex, parent) } @@ -567,6 +571,85 @@ private[parquet] class CatalystNativeArrayConverter( } } +/** + * A `parquet.io.api.GroupConverter` that converts a single-element groups that + * match the characteristics of an array contains null (see + * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an + * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * + * @param elementType The type of the array elements (complex or primitive) + * @param index The position of this (array) field inside its parent converter + * @param parent The parent converter + * @param buffer A data buffer + */ +private[parquet] class CatalystArrayContainsNullConverter( + val elementType: DataType, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var buffer: Buffer[Any]) + extends CatalystConverter { + + def this(elementType: DataType, index: Int, parent: CatalystConverter) = + this( + elementType, + index, + parent, + new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) + + protected[parquet] val converter: Converter = new CatalystConverter { + + private var current: Any = null + + val converter = CatalystConverter.createConverter( + new CatalystConverter.FieldType( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + elementType, + false), + fieldIndex = 0, + parent = this) + + override def getConverter(fieldIndex: Int): Converter = converter + + override def end(): Unit = parent.updateField(index, current) + + override def start(): Unit = { + current = null + } + + override protected[parquet] val size: Int = 1 + override protected[parquet] val index: Int = 0 + override protected[parquet] val parent = CatalystArrayContainsNullConverter.this + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + current = value + } + + override protected[parquet] def clearBuffer(): Unit = {} + } + + override def getConverter(fieldIndex: Int): Converter = converter + + // arrays have only one (repeated) field, which is its elements + override val size = 1 + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + buffer += value + } + + override protected[parquet] def clearBuffer(): Unit = { + buffer.clear() + } + + override def start(): Unit = {} + + override def end(): Unit = { + assert(parent != null) + // here we need to make sure to use ArrayScalaType + parent.updateField(index, buffer.toArray.toSeq) + clearBuffer() + } +} + /** * This converter is for multi-element groups of primitive or complex types * that have repetition level optional or required (so struct fields). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 6a657c20fe46c..bdf02401b21be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -173,7 +173,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null) { schema match { - case t @ ArrayType(_, false) => writeArray( + case t @ ArrayType(_, _) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) case t @ MapType(_, _, _) => writeMap( @@ -228,45 +228,57 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeArray( schema: ArrayType, array: CatalystConverter.ArrayScalaType[_]): Unit = { val elementType = schema.elementType writer.startGroup() if (array.size > 0) { - writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) - var i = 0 - while(i < array.size) { - writeValue(elementType, array(i)) - i = i + 1 + if (schema.containsNull) { + writer.startField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writer.startGroup() + if (array(i) != null) { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + writeValue(elementType, array(i)) + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + } + writer.endGroup() + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + } else { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writeValue(elementType, array(i)) + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } - writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } writer.endGroup() } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeMap( schema: MapType, map: CatalystConverter.MapScalaType[_, _]): Unit = { writer.startGroup() if (map.size > 0) { writer.startField(CatalystConverter.MAP_SCHEMA_NAME, 0) - writer.startGroup() - writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - for(key <- map.keys) { + for ((key, value) <- map) { + writer.startGroup() + writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) writeValue(schema.keyType, key) + writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) + if (value != null) { + writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + writeValue(schema.valueType, value) + writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + } + writer.endGroup() } - writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - for(value <- map.values) { - writeValue(schema.valueType, value) - } - writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - writer.endGroup() writer.endField(CatalystConverter.MAP_SCHEMA_NAME, 0) } writer.endGroup() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index af8cd0a73b674..1a52377651737 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -119,7 +119,13 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } case ParquetOriginalType.MAP => { assert( @@ -129,28 +135,32 @@ private[parquet] object ParquetTypesConverter extends Logging { assert( keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } case _ => { // Note: the order of these checks is important! if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } else if (correspondsToArray(groupType)) { // ArrayType - val elementType = toDataType(groupType.getFields.apply(0), isBinaryAsString) - ArrayType(elementType, containsNull = false) + val field = groupType.getFields.apply(0) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } else { // everything else: StructType val fields = groupType .getFields @@ -249,13 +259,27 @@ private[parquet] object ParquetTypesConverter extends Logging { inArray = true) ConversionPatterns.listType(repetition, name, parquetElementType) } + case ArrayType(elementType, true) => { + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = true, + inArray = false) + ConversionPatterns.listType( + repetition, + name, + new ParquetGroupType( + Repetition.REPEATED, + CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, + parquetElementType)) + } case StructType(structFields) => { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } new ParquetGroupType(repetition, name, fields) } - case MapType(keyType, valueType, _) => { + case MapType(keyType, valueType, valueContainsNull) => { val parquetKeyType = fromDataType( keyType, @@ -266,7 +290,7 @@ private[parquet] object ParquetTypesConverter extends Logging { fromDataType( valueType, CatalystConverter.MAP_VALUE_SCHEMA_NAME, - nullable = false, + nullable = valueContainsNull, inArray = false) ConversionPatterns.mapType( repetition, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 28f43b36832ac..4219cc080000b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -78,7 +78,9 @@ case class AllDataTypesWithNonPrimitiveType( booleanField: Boolean, binaryField: Array[Byte], array: Seq[Int], - map: Map[Int, String], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapValueContainsNull: Map[Int, Option[Long]], data: Data) class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { @@ -287,7 +289,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, (0 to x).map(_.toByte).toArray, - (0 until x), (0 until x).map(i => i -> s"$i").toMap, Data((0 until x), Nested(x, s"$x")))) + (0 until x), + (0 until x).map(Option(_).filter(_ % 3 == 0)), + (0 until x).map(i => i -> i.toLong).toMap, + (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), + Data((0 until x), Nested(x, s"$x")))) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -302,8 +308,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getBoolean(7) === (i % 2 == 0)) assert(result(i)(8) === (0 to i).map(_.toByte).toArray) assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => i -> s"$i").toMap) - assert(result(i)(11) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) + assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) + assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) + assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) + assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) } } From 7726e566cf9e2e8cf65641df056a42beff30be63 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 26 Aug 2014 18:55:00 -0700 Subject: [PATCH 114/192] Fix unclosed HTML tag in Yarn docs. --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9bc20dbf926b2..943f06b114cb9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -75,7 +75,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes (none) Comma-separated list of files to be placed in the working directory of each executor. - + spark.yarn.executor.memoryOverhead From 2381e90dc04932f10d4c4534dcb64168799dc8af Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Aug 2014 21:59:48 -0700 Subject: [PATCH 115/192] [SPARK-3224] FetchFailed reduce stages should only show up once in failed stages (in UI) This is a HOTFIX for 1.1. Author: Reynold Xin Author: Kay Ousterhout Closes #2127 from rxin/SPARK-3224 and squashes the following commits: effb1ce [Reynold Xin] Move log message. 49282b3 [Reynold Xin] Kay's feedback. 3f01847 [Reynold Xin] Merge pull request #2 from kayousterhout/SPARK-3224 796d282 [Kay Ousterhout] Added unit test for SPARK-3224 3d3d356 [Reynold Xin] Remove map output loc even for repeated FetchFaileds. 1dd3eb5 [Reynold Xin] [SPARK-3224] FetchFailed reduce stages should only show up once in the failed stages UI. (cherry picked from commit bf719056b71d55e1194554661dfa194ed03d364d) Signed-off-by: Patrick Wendell --- .../apache/spark/scheduler/DAGScheduler.scala | 32 +++++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 41 ++++++++++++++++++- 2 files changed, 59 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 34131984570e4..2ccc27324ac8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1045,31 +1045,39 @@ class DAGScheduler( stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - markStageAsFinished(failedStage, Some("Fetch failure")) - runningStages -= failedStage - // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.name + - ") for resubmision due to a fetch failure") - // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) - if (mapId != -1) { - mapStage.removeOutputLoc(mapId, bmAddress) - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + + // It is likely that we receive multiple FetchFailed for a single stage (because we have + // multiple tasks running concurrently on different executors). In that case, it is possible + // the fetch failure has already been handled by the scheduler. + if (runningStages.contains(failedStage)) { + logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + + s"due to a fetch failure from $mapStage (${mapStage.name})") + markStageAsFinished(failedStage, Some("Fetch failure")) + runningStages -= failedStage } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + - "); marking it for resubmission") + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. + // TODO: Cancel running tasks in the stage import env.actorSystem.dispatcher + logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + + s"$failedStage (${failedStage.name}) due to fetch failure") env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } failedStages += failedStage failedStages += mapStage + + // Mark the map whose fetch failed as broken in the map stage + if (mapId != -1) { + mapStage.removeOutputLoc(mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + } + // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bd829752eb401..f5fed988ade24 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map} import scala.language.reflectiveCalls import akka.actor._ @@ -98,7 +98,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val WAIT_TIMEOUT_MILLIS = 10000 val sparkListener = new SparkListener() { val successfulStages = new HashSet[Int]() - val failedStages = new HashSet[Int]() + val failedStages = new ArrayBuffer[Int]() override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo if (stageInfo.failureReason.isEmpty) { @@ -435,6 +435,43 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("trivial shuffle with multiple fetch failures") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + // The MapOutputTracker should know about both map output locations. + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) + + // The first result task fails, with a fetch failure for the output from the first mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), + null, + Map[Long, Any](), + null, + null)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + + // The second ResultTask fails, with a fetch failure for the output from the second mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1), + null, + Map[Long, Any](), + null, + null)) + // The SparkListener should not receive redundant failure events. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.size == 1) + } + test("ignore late map task completions") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From e7672f19674c37fbd1a43fb3793b69097349bca1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 26 Aug 2014 23:06:11 -0700 Subject: [PATCH 116/192] [SPARK-3167] Handle special driver configs in Windows (Branch 1.1) This is an effort to bring the Windows scripts up to speed after recent splashing changes in #1845. Author: Andrew Or Closes #2156 from andrewor14/windows-config-branch-1.1 and squashes the following commits: 00b9dfe [Andrew Or] [SPARK-3167] Handle special driver configs in Windows --- bin/compute-classpath.cmd | 3 +- bin/spark-class2.cmd | 46 ++++++++++++++++--- bin/spark-submit | 2 +- bin/spark-submit.cmd | 34 +++++++++----- .../SparkSubmitDriverBootstrapper.scala | 19 +++++--- python/pyspark/java_gateway.py | 17 +++++++ 6 files changed, 95 insertions(+), 26 deletions(-) mode change 100755 => 100644 bin/spark-class2.cmd diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 58710cd1bd548..5ad52452a5c98 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,8 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf + if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd old mode 100755 new mode 100644 index e420eb409e529..b60625554f31f --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! + setlocal enabledelayedexpansion set SCALA_VERSION=2.10 @@ -38,7 +40,7 @@ if not "x%1"=="x" goto arg_given if not "x%SPARK_MEM%"=="x" ( echo Warning: SPARK_MEM is deprecated, please use a more specific config option - echo e.g., spark.executor.memory or SPARK_DRIVER_MEMORY. + echo e.g., spark.executor.memory or spark.driver.memory. ) rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -67,18 +69,26 @@ rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% -rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. -) else if "%1"=="org.apache.spark.repl.Main" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_REPL_OPTS% +rem Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + +rem SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. +rem The repl also uses SPARK_REPL_OPTS. +) else if "%1"=="org.apache.spark.deploy.SparkSubmit" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_SUBMIT_OPTS% %SPARK_REPL_OPTS% + if not "x%SPARK_SUBMIT_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_SUBMIT_LIBRARY_PATH% + ) else if not "x%SPARK_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_LIBRARY_PATH% + ) if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% + if not "x%SPARK_SUBMIT_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_SUBMIT_DRIVER_MEMORY% ) else ( set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% ) -rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! +rem Set JAVA_OPTS to be able to load native libraries and to set heap size +set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% rem Test whether the user has built Spark if exist "%FWDIR%RELEASE" goto skip_build_test @@ -109,5 +119,27 @@ rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +rem In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. +rem Here we must parse the properties file for relevant "spark.driver.*" configs before launching +rem the driver JVM itself. Instead of handling this complexity here, we launch a separate JVM +rem to prepare the launch environment of this driver JVM. + +rem In this case, leave out the main class (org.apache.spark.deploy.SparkSubmit) and use our own. +rem Leaving out the first argument is surprisingly difficult to do in Windows. Note that this must +rem be done here because the Windows "shift" command does not work in a conditional block. +set BOOTSTRAP_ARGS= +shift +:start_parse +if "%~1" == "" goto end_parse +set BOOTSTRAP_ARGS=%BOOTSTRAP_ARGS% %~1 +shift +goto start_parse +:end_parse + +if not [%SPARK_SUBMIT_BOOTSTRAP_DRIVER%] == [] ( + set SPARK_CLASS=1 + "%RUNNER%" org.apache.spark.deploy.SparkSubmitDriverBootstrapper %BOOTSTRAP_ARGS% +) else ( + "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +) :exit diff --git a/bin/spark-submit b/bin/spark-submit index 32c911cd0438b..277c4ce571ca2 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,7 +17,7 @@ # limitations under the License. # -# NOTE: Any changes in this file must be reflected in SparkClassLauncher.scala! +# NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=("$@") diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 6eb702ed8c561..cf6046d1547ad 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -17,23 +17,28 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! + set SPARK_HOME=%~dp0.. set ORIG_ARGS=%* -rem Clear the values of all variables used -set DEPLOY_MODE= -set DRIVER_MEMORY= +rem Reset the values of all variables used +set SPARK_SUBMIT_DEPLOY_MODE=client +set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf +set SPARK_SUBMIT_DRIVER_MEMORY= set SPARK_SUBMIT_LIBRARY_PATH= set SPARK_SUBMIT_CLASSPATH= set SPARK_SUBMIT_OPTS= -set SPARK_DRIVER_MEMORY= +set SPARK_SUBMIT_BOOTSTRAP_DRIVER= :loop if [%1] == [] goto continue if [%1] == [--deploy-mode] ( - set DEPLOY_MODE=%2 + set SPARK_SUBMIT_DEPLOY_MODE=%2 + ) else if [%1] == [--properties-file] ( + set SPARK_SUBMIT_PROPERTIES_FILE=%2 ) else if [%1] == [--driver-memory] ( - set DRIVER_MEMORY=%2 + set SPARK_SUBMIT_DRIVER_MEMORY=%2 ) else if [%1] == [--driver-library-path] ( set SPARK_SUBMIT_LIBRARY_PATH=%2 ) else if [%1] == [--driver-class-path] ( @@ -45,12 +50,19 @@ if [%1] == [] goto continue goto loop :continue -if [%DEPLOY_MODE%] == [] ( - set DEPLOY_MODE=client -) +rem For client mode, the driver will be launched in the same JVM that launches +rem SparkSubmit, so we may need to read the properties file for any extra class +rem paths, library paths, java options and memory early on. Otherwise, it will +rem be too late by the time the driver JVM has started. -if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( - set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( + if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( + rem Parse the properties file only if the special configs exist + for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ + %SPARK_SUBMIT_PROPERTIES_FILE%') do ( + set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + ) + ) ) cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index af607e6a4a065..7ca96ed57c2db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -133,17 +133,24 @@ private[spark] object SparkSubmitDriverBootstrapper { val process = builder.start() // Redirect stdin, stdout, and stderr to/from the child JVM - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") - stdinThread.start() stdoutThread.start() stderrThread.start() - // Terminate on broken pipe, which signals that the parent process has exited. This is - // important for the PySpark shell, where Spark submit itself is a python subprocess. - stdinThread.join() - process.destroy() + // In Windows, the subprocess reads directly from our stdin, so we should avoid spawning + // a thread that contends with the subprocess in reading from System.in. + if (Utils.isWindows) { + // For the PySpark shell, the termination of this process is handled in java_gateway.py + process.waitFor() + } else { + // Terminate on broken pipe, which signals that the parent process has exited. This is + // important for the PySpark shell, where Spark submit itself is a python subprocess. + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + stdinThread.start() + stdinThread.join() + process.destroy() + } } } diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6f4f62f23bc4d..9c70fa5c16d0c 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -15,6 +15,7 @@ # limitations under the License. # +import atexit import os import sys import signal @@ -69,6 +70,22 @@ def preexec_func(): error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) + # In Windows, ensure the Java child processes do not linger after Python has exited. + # In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when + # the parent process' stdin sends an EOF). In Windows, however, this is not possible + # because java.lang.Process reads directly from the parent process' stdin, contending + # with any opportunity to read an EOF from the parent. Note that this is only best + # effort and will not take effect if the python process is violently terminated. + if on_windows: + # In Windows, the child process here is "spark-submit.cmd", not the JVM itself + # (because the UNIX "exec" command is not available). This means we cannot simply + # call proc.kill(), which kills only the "spark-submit.cmd" process but not the + # JVMs. Instead, we use "taskkill" with the tree-kill option "/t" to terminate all + # child processes in the tree (http://technet.microsoft.com/en-us/library/bb491009.aspx) + def killChild(): + Popen(["cmd", "/c", "taskkill", "/f", "/t", "/pid", str(proc.pid)]) + atexit.register(killChild) + # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: class EchoOutputThread(Thread): From 6f82a4b13a2583ef1c0a9523832333a8a61a28d5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 26 Aug 2014 23:40:50 -0700 Subject: [PATCH 117/192] HOTFIX: Minor typo in conf template --- conf/spark-defaults.conf.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index 94427029b94d7..a48dcc70e1363 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -7,4 +7,4 @@ # spark.eventLog.dir hdfs://namenode:8021/directory # spark.serializer org.apache.spark.serializer.KryoSerializer # spark.driver.memory 5g -# spark.executor.extraJavaOptions -XX:+PrintGCDetail -Dkey=value -Dnumbers="one two three" +# spark.executor.extraJavaOptions -XX:+PrintGCDetails -Dkey=value -Dnumbers="one two three" From 5cf1e440137006eedd6846ac8fa57ccf9fd1958d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 27 Aug 2014 00:13:38 -0700 Subject: [PATCH 118/192] [SPARK-3139] Made ContextCleaner to not block on shuffles As a workaround for SPARK-3015, the ContextCleaner was made "blocking", that is, it cleaned items one-by-one. But shuffles can take a long time to be deleted. Given that the RC for 1.1 is imminent, this PR makes a narrow change in the context cleaner - not wait for shuffle cleanups to complete. Also it changes the error messages on failure to delete to be milder warnings, as exceptions in the delete code path for one item does not really stop the actual functioning of the system. Author: Tathagata Das Closes #2143 from tdas/cleaner-shuffle-fix and squashes the following commits: 9c84202 [Tathagata Das] Restoring default blocking behavior in ContextCleanerSuite, and added docs to identify that spark.cleaner.referenceTracking.blocking does not control shuffle. 2181329 [Tathagata Das] Mark shuffle cleanup as non-blocking. e337cc2 [Tathagata Das] Changed semantics based on PR comments. 387b578 [Tathagata Das] Made ContextCleaner to not block on shuffles (cherry picked from commit 3e2864e40472b32e6a7eec5ba3bc83562d2a1a62) Signed-off-by: Patrick Wendell --- .../org/apache/spark/ContextCleaner.scala | 18 ++++++++++++++++-- .../spark/storage/BlockManagerMaster.scala | 12 +++++++----- .../org/apache/spark/ContextCleanerSuite.scala | 3 +++ 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 3848734d6f639..ede1e23f4fcc5 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -65,7 +65,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val cleaningThread = new Thread() { override def run() { keepCleaning() }} /** - * Whether the cleaning thread will block on cleanup tasks. + * Whether the cleaning thread will block on cleanup tasks (other than shuffle, which + * is controlled by the `spark.cleaner.referenceTracking.blocking.shuffle` parameter). * * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary * workaround for the issue, which is ultimately caused by the way the BlockManager actors @@ -76,6 +77,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val blockOnCleanupTasks = sc.conf.getBoolean( "spark.cleaner.referenceTracking.blocking", true) + /** + * Whether the cleaning thread will block on shuffle cleanup tasks. + * + * When context cleaner is configured to block on every delete request, it can throw timeout + * exceptions on cleanup of shuffle blocks, as reported in SPARK-3139. To avoid that, this + * parameter by default disables blocking on shuffle cleanups. Note that this does not affect + * the cleanup of RDDs and broadcasts. This is intended to be a temporary workaround, + * until the real Akka issue (referred to in the comment above `blockOnCleanupTasks`) is + * resolved. + */ + private val blockOnShuffleCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking.shuffle", false) + @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ @@ -128,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { case CleanRDD(rddId) => doCleanupRDD(rddId, blocking = blockOnCleanupTasks) case CleanShuffle(shuffleId) => - doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 669307765d1fa..e67b3dc5ce02e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -101,7 +101,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) future.onFailure { - case e: Throwable => logError("Failed to remove RDD " + rddId, e) + case e: Exception => + logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -112,7 +113,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeShuffle(shuffleId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) future.onFailure { - case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + case e: Exception => + logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -124,9 +126,9 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log val future = askDriverWithReply[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) future.onFailure { - case e: Throwable => - logError("Failed to remove broadcast " + broadcastId + - " with removeFromMaster = " + removeFromMaster, e) + case e: Exception => + logWarning(s"Failed to remove broadcast $broadcastId" + + s" with removeFromMaster = $removeFromMaster - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 4bc4346c0a288..2744894277ae8 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -52,6 +52,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha .setMaster("local[2]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) before { @@ -243,6 +244,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) @@ -319,6 +321,7 @@ class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[Sor .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) From ca01de1b98ae17d9f85dbd07e3546c985061c8a5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 27 Aug 2014 00:59:23 -0700 Subject: [PATCH 119/192] [SPARK-3237][SQL] Fix parquet filters with UDFs Author: Michael Armbrust Closes #2153 from marmbrus/parquetFilters and squashes the following commits: 712731a [Michael Armbrust] Use closure serializer for sending filters. 1e83f80 [Michael Armbrust] Clean udf functions. (cherry picked from commit e1139dd60e0692e8adb1337c1f605165ce4b8895) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 4 ++++ .../org/apache/spark/sql/parquet/ParquetFilters.scala | 8 ++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 63ac2a608b6ff..0b3c1df453fb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -18,10 +18,14 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.util.ClosureCleaner case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { + // Clean function when not called with default no-arg constructor. + if (function != null) { ClosureCleaner.clean(function) } + type EvaluatedType = Any def nullable = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 2298a9b933df5..fe28e0d7269e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.nio.ByteBuffer + import org.apache.hadoop.conf.Configuration import parquet.filter._ @@ -25,6 +27,7 @@ import parquet.column.ColumnReader import com.google.common.io.BaseEncoding +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} import org.apache.spark.sql.catalyst.expressions._ @@ -237,7 +240,8 @@ object ParquetFilters { */ def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = { if (filters.length > 0) { - val serialized: Array[Byte] = SparkSqlSerializer.serialize(filters) + val serialized: Array[Byte] = + SparkEnv.get.closureSerializer.newInstance().serialize(filters).array() val encoded: String = BaseEncoding.base64().encode(serialized) conf.set(PARQUET_FILTER_DATA, encoded) } @@ -252,7 +256,7 @@ object ParquetFilters { val data = conf.get(PARQUET_FILTER_DATA) if (data != null) { val decoded: Array[Byte] = BaseEncoding.base64().decode(data) - SparkSqlSerializer.deserialize(decoded) + SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(decoded)) } else { Seq() } From 74012475bf19ceb5caca2eaa90b7c8e3fdfaaf8f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 27 Aug 2014 01:19:48 -0700 Subject: [PATCH 120/192] [SPARK-2830][MLLIB] doc update for 1.1 1. renamed mllib-basics to mllib-data-types 1. renamed mllib-stats to mllib-statistics 1. moved random data generation to the bottom of mllib-stats 1. updated toc accordingly atalwalkar Author: Xiangrui Meng Closes #2151 from mengxr/mllib-doc-1.1 and squashes the following commits: 0bd79f3 [Xiangrui Meng] add mllib-data-types b64a5d7 [Xiangrui Meng] update the content list of basis statistics in mllib-guide f625cc2 [Xiangrui Meng] move mllib-basics to mllib-data-types 4d69250 [Xiangrui Meng] move random data generation to the bottom of statistics e64f3ce [Xiangrui Meng] move mllib-stats.md to mllib-statistics.md (cherry picked from commit 43dfc84f883822ea27b6e312d4353bf301c2e7ef) Signed-off-by: Xiangrui Meng --- docs/{mllib-basics.md => mllib-data-types.md} | 4 +- docs/mllib-dimensionality-reduction.md | 4 +- docs/mllib-guide.md | 9 +- docs/{mllib-stats.md => mllib-statistics.md} | 156 +++++++++--------- 4 files changed, 87 insertions(+), 86 deletions(-) rename docs/{mllib-basics.md => mllib-data-types.md} (99%) rename docs/{mllib-stats.md => mllib-statistics.md} (99%) diff --git a/docs/mllib-basics.md b/docs/mllib-data-types.md similarity index 99% rename from docs/mllib-basics.md rename to docs/mllib-data-types.md index 8752df412950a..101dc2f8695f3 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-data-types.md @@ -1,7 +1,7 @@ --- layout: global -title: Basics - MLlib -displayTitle: MLlib - Basics +title: Data Types - MLlib +displayTitle: MLlib - Data Types --- * Table of contents diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 9f2cf6d48ec75..21cb35b4270ca 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on the RowMatrix class. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -58,7 +58,7 @@ passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. ### SVD Example MLlib provides SVD functionality to row-oriented matrices, provided in the -RowMatrix class. +RowMatrix class.
    diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4d4198b9e0452..d3a510b3c17c6 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -7,12 +7,13 @@ MLlib is Spark's scalable machine learning library consisting of common learning including classification, regression, clustering, collaborative filtering, dimensionality reduction, as well as underlying optimization primitives, as outlined below: -* [Data types](mllib-basics.html) -* [Basic statistics](mllib-stats.html) - * random data generation - * stratified sampling +* [Data types](mllib-data-types.html) +* [Basic statistics](mllib-statistics.html) * summary statistics + * correlations + * stratified sampling * hypothesis testing + * random data generation * [Classification and regression](mllib-classification-regression.html) * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) * [decision trees](mllib-decision-tree.html) diff --git a/docs/mllib-stats.md b/docs/mllib-statistics.md similarity index 99% rename from docs/mllib-stats.md rename to docs/mllib-statistics.md index 511a9fbf710cc..c4632413991f1 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-statistics.md @@ -1,7 +1,7 @@ --- layout: global -title: Statistics Functionality - MLlib -displayTitle: MLlib - Statistics Functionality +title: Basic Statistics - MLlib +displayTitle: MLlib - Basic Statistics --- * Table of contents @@ -25,7 +25,7 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` -## Summary Statistics +## Summary statistics We provide column summary statistics for `RDD[Vector]` through the function `colStats` available in `Statistics`. @@ -104,81 +104,7 @@ print summary.numNonzeros()
    -## Random data generation - -Random data generation is useful for randomized algorithms, prototyping, and performance testing. -MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: -uniform, standard normal, or Poisson. - -
    -
    -[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.random.RandomRDDs._ - -val sc: SparkContext = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -val u = normalRDD(sc, 1000000L, 10) -// Apply a transform to get a random double RDD following `N(1, 4)`. -val v = u.map(x => 1.0 + 2.0 * x) -{% endhighlight %} -
    - -
    -[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight java %} -import org.apache.spark.SparkContext; -import org.apache.spark.api.JavaDoubleRDD; -import static org.apache.spark.mllib.random.RandomRDDs.*; - -JavaSparkContext jsc = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); -// Apply a transform to get a random double RDD following `N(1, 4)`. -JavaDoubleRDD v = u.map( - new Function() { - public Double call(Double x) { - return 1.0 + 2.0 * x; - } - }); -{% endhighlight %} -
    - -
    -[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight python %} -from pyspark.mllib.random import RandomRDDs - -sc = ... # SparkContext - -# Generate a random double RDD that contains 1 million i.i.d. values drawn from the -# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -u = RandomRDDs.uniformRDD(sc, 1000000L, 10) -# Apply a transform to get a random double RDD following `N(1, 4)`. -v = u.map(lambda x: 1.0 + 2.0 * x) -{% endhighlight %} -
    - -
    - -## Correlations calculation +## Correlations Calculating the correlation between two series of data is a common operation in Statistics. In MLlib we provide the flexibility to calculate pairwise correlations among many series. The supported @@ -455,3 +381,77 @@ for (ChiSqTestResult result : featureTestResults) {
    + +## Random data generation + +Random data generation is useful for randomized algorithms, prototyping, and performance testing. +MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +uniform, standard normal, or Poisson. + +
    +
    +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.random.RandomRDDs._ + +val sc: SparkContext = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +val u = normalRDD(sc, 1000000L, 10) +// Apply a transform to get a random double RDD following `N(1, 4)`. +val v = u.map(x => 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight java %} +import org.apache.spark.SparkContext; +import org.apache.spark.api.JavaDoubleRDD; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +JavaSparkContext jsc = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); +// Apply a transform to get a random double RDD following `N(1, 4)`. +JavaDoubleRDD v = u.map( + new Function() { + public Double call(Double x) { + return 1.0 + 2.0 * x; + } + }); +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight python %} +from pyspark.mllib.random import RandomRDDs + +sc = ... # SparkContext + +# Generate a random double RDD that contains 1 million i.i.d. values drawn from the +# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +# Apply a transform to get a random double RDD following `N(1, 4)`. +v = u.map(lambda x: 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    From 7286d5707af69d0acfc010f0458626c30f6aef0a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 27 Aug 2014 01:45:59 -0700 Subject: [PATCH 121/192] [SPARK-3227] [mllib] Added migration guide for v1.0 to v1.1 The only updates are in DecisionTree. CC: mengxr Author: Joseph K. Bradley Closes #2146 from jkbradley/mllib-migration and squashes the following commits: 5a1f487 [Joseph K. Bradley] small edit to doc 411d6d9 [Joseph K. Bradley] Added migration guide for v1.0 to v1.1. The only updates are in DecisionTree. (cherry picked from commit 171a41cb034f4ea80f6a3c91a6872970de16a14a) Signed-off-by: Xiangrui Meng --- docs/mllib-guide.md | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index d3a510b3c17c6..94fc98ce4fabe 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -60,6 +60,32 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 # Migration Guide +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + ## From 0.9 to 1.0 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few @@ -85,7 +111,7 @@ val vector: Vector = Vectors.dense(array) // a dense vector [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. -*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. +*Note*: Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`.
    From 1d468df33c7b8680af12fcdb66ed91f48c80cae3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 Aug 2014 02:39:02 -0700 Subject: [PATCH 122/192] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. Currently lot of errors get thrown from Avro IPC layer when the dstream or sink is shutdown. This PR cleans it up. Some refactoring is done in the receiver code to put all of the RPC code into a single Try and just recover from that. The sink code has also been cleaned up. Author: Hari Shreedharan Closes #2065 from harishreedharan/clean-flume-shutdown and squashes the following commits: f93a07c [Hari Shreedharan] Formatting fixes. d7427cc [Hari Shreedharan] More fixes! a0a8852 [Hari Shreedharan] Fix race condition, hopefully! Minor other changes. 4c9ed02 [Hari Shreedharan] Remove unneeded list in Callback handler. Other misc changes. 8fee36f [Hari Shreedharan] Scala-library is required, else maven build fails. Also catch InterruptedException in TxnProcessor. 445e700 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown 87232e0 [Hari Shreedharan] Refactor Flume Input Stream. Clean up code, better error handling. 9001d26 [Hari Shreedharan] Change log level to debug in TransactionProcessor#shutdown method e7b8d82 [Hari Shreedharan] Incorporate review feedback 598efa7 [Hari Shreedharan] Clean up some exception handling code e1027c6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown ed608c8 [Hari Shreedharan] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. (cherry picked from commit 6f671d04fa98f97fd48c5e749b9f47dd4a8b4f44) Signed-off-by: Tathagata Das --- external/flume-sink/pom.xml | 4 + .../flume/sink/SparkAvroCallbackHandler.scala | 56 ++++-- .../flume/sink/TransactionProcessor.scala | 18 +- .../streaming/flume/FlumeBatchFetcher.scala | 167 ++++++++++++++++++ .../flume/FlumePollingInputDStream.scala | 77 ++------ 5 files changed, 236 insertions(+), 86 deletions(-) create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index a297459f1658e..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -70,6 +70,10 @@ scalatest_${scala.binary.version} test + + org.scala-lang + scala-library + - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-snapshot2 + HEAD @@ -880,7 +881,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From d4cf7a068da099f0f07f04a834d7edf6b743ceb3 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Wed, 27 Aug 2014 15:50:30 -0700 Subject: [PATCH 138/192] Add line continuation for script to work w/ py2.7.5 Error was - $ SPARK_HOME=$PWD/dist ./dev/create-release/generate-changelist.py File "./dev/create-release/generate-changelist.py", line 128 if day < SPARK_REPO_CHANGE_DATE1 or ^ SyntaxError: invalid syntax Author: Matthew Farrellee Closes #2139 from mattf/master-fix-generate-changelist.py-0 and squashes the following commits: 6b3a900 [Matthew Farrellee] Add line continuation for script to work w/ py2.7.5 (cherry picked from commit 64d8ecbbe94c47236ff2d8c94d7401636ba6fca4) Signed-off-by: Patrick Wendell --- dev/create-release/generate-changelist.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py index de1b5d4ae1314..2e1a35a629342 100755 --- a/dev/create-release/generate-changelist.py +++ b/dev/create-release/generate-changelist.py @@ -125,8 +125,8 @@ def cleanup(ask=True): pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] github_url = "github.com/apache/spark/pull/%s" % pr_num day = time.strptime(date.split()[0], "%Y-%m-%d") - if day < SPARK_REPO_CHANGE_DATE1 or - (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH): + if (day < SPARK_REPO_CHANGE_DATE1 or + (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH)): github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num append_to_changelist(" %s" % subject) From 8597e9cf356b0d8e17600a49efc4c4a0356ecb5d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 27 Aug 2014 15:55:59 -0700 Subject: [PATCH 139/192] BUILD: Updating CHANGES.txt for Spark 1.1 --- CHANGES.txt | 14470 ++++++++++++++++++++ dev/create-release/generate-changelist.py | 4 +- 2 files changed, 14472 insertions(+), 2 deletions(-) create mode 100644 CHANGES.txt diff --git a/CHANGES.txt b/CHANGES.txt new file mode 100644 index 0000000000000..6efb0225740a6 --- /dev/null +++ b/CHANGES.txt @@ -0,0 +1,14470 @@ +Spark Change Log +---------------- + +Release 1.1.0 + + Add line continuation for script to work w/ py2.7.5 + Matthew Farrellee + 2014-08-27 15:50:30 -0700 + Commit: d4cf7a0, github.com/apache/spark/pull/2139 + + [SPARK-3235][SQL] Ensure in-memory tables don't always broadcast. + Michael Armbrust + 2014-08-27 15:14:08 -0700 + Commit: 9a62cf3, github.com/apache/spark/pull/2147 + + [SPARK-3065][SQL] Add locale setting to fix results do not match for udf_unix_timestamp format "yyyy MMM dd h:mm:ss a" run with not "America/Los_Angeles" TimeZone in HiveCompatibilitySuite + luogankun + 2014-08-27 15:08:22 -0700 + Commit: 5ea260e, github.com/apache/spark/pull/1968 + + [SQL] [SPARK-3236] Reading Parquet tables from Metastore mangles location + Aaron Davidson + 2014-08-27 15:05:47 -0700 + Commit: 7711687, github.com/apache/spark/pull/2150 + + [SPARK-3252][SQL] Add missing condition for test + viirya + 2014-08-27 14:55:05 -0700 + Commit: b3d763b, github.com/apache/spark/pull/2159 + + [SPARK-3243] Don't use stale spark-driver.* system properties + Andrew Or + 2014-08-27 14:46:56 -0700 + Commit: c1ffa3e, github.com/apache/spark/pull/2154 + + Spark-3213 Fixes issue with spark-ec2 not detecting slaves created with "Launch More like this" + Vida Ha + 2014-08-27 14:26:06 -0700 + Commit: 3cb4e17, github.com/apache/spark/pull/2163 + + [SPARK-3138][SQL] sqlContext.parquetFile should be able to take a single file as parameter + chutium + 2014-08-27 13:13:04 -0700 + Commit: 90f8f3e, github.com/apache/spark/pull/2044 + + [SPARK-3197] [SQL] Reduce the Expression tree object creations for aggregation function (min/max) + Cheng Hao + 2014-08-27 12:50:47 -0700 + Commit: 4c7f082, github.com/apache/spark/pull/2113 + + [SPARK-3118][SQL]add "SHOW TBLPROPERTIES tblname;" and "SHOW COLUMNS (FROM|IN) table_name [(FROM|IN) db_name]" support + u0jing + 2014-08-27 12:47:14 -0700 + Commit: 19cda07, github.com/apache/spark/pull/2034 + + SPARK-3259 - User data should be given to the master + Allan Douglas R. de Oliveira + 2014-08-27 12:43:22 -0700 + Commit: 0c94a5b, github.com/apache/spark/pull/2162 + + [SPARK-2608][Core] Fixed command line option passing issue over Mesos via SPARK_EXECUTOR_OPTS + Cheng Lian + 2014-08-27 12:39:21 -0700 + Commit: 935bffe, github.com/apache/spark/pull/2161 + + [SPARK-3239] [PySpark] randomize the dirs for each process + Davies Liu + 2014-08-27 10:40:35 -0700 + Commit: 092121e, github.com/apache/spark/pull/2152 + + [SPARK-3170][CORE][BUG]:RDD info loss in "StorageTab" and "ExecutorTab" + uncleGen + 2014-08-27 10:32:13 -0700 + Commit: 8f8e2a4, github.com/apache/spark/pull/2131 + + [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. + Hari Shreedharan + 2014-08-27 02:39:02 -0700 + Commit: 1d468df, github.com/apache/spark/pull/2065 + + [SPARK-3227] [mllib] Added migration guide for v1.0 to v1.1 + Joseph K. Bradley + 2014-08-27 01:45:59 -0700 + Commit: 7286d57, github.com/apache/spark/pull/2146 + + [SPARK-2830][MLLIB] doc update for 1.1 + Xiangrui Meng + 2014-08-27 01:19:48 -0700 + Commit: 7401247, github.com/apache/spark/pull/2151 + + [SPARK-3237][SQL] Fix parquet filters with UDFs + Michael Armbrust + 2014-08-27 00:59:23 -0700 + Commit: ca01de1, github.com/apache/spark/pull/2153 + + [SPARK-3139] Made ContextCleaner to not block on shuffles + Tathagata Das + 2014-08-27 00:13:38 -0700 + Commit: 5cf1e44, github.com/apache/spark/pull/2143 + + HOTFIX: Minor typo in conf template + Patrick Wendell + 2014-08-26 23:40:50 -0700 + Commit: 6f82a4b + + [SPARK-3167] Handle special driver configs in Windows (Branch 1.1) + Andrew Or + 2014-08-26 23:06:11 -0700 + Commit: e7672f1, github.com/apache/spark/pull/2156 + + [SPARK-3224] FetchFailed reduce stages should only show up once in failed stages (in UI) + Reynold Xin , Kay Ousterhout + 2014-08-26 21:59:48 -0700 + Commit: 2381e90, github.com/apache/spark/pull/2127 + + Fix unclosed HTML tag in Yarn docs. + Josh Rosen + 2014-08-26 18:55:00 -0700 + Commit: 7726e56 + + [SPARK-3036][SPARK-3037][SQL] Add MapType/ArrayType containing null value support to Parquet. + Takuya UESHIN + 2014-08-26 18:28:41 -0700 + Commit: 8b5af6f, github.com/apache/spark/pull/2032 + + [Docs] Run tests like in contributing guide + nchammas + 2014-08-26 17:50:04 -0700 + Commit: 0d97233, github.com/apache/spark/pull/2149 + + [SPARK-2964] [SQL] Remove duplicated code from spark-sql and start-thriftserver.sh + Cheng Lian , Kousuke Saruta + 2014-08-26 17:33:40 -0700 + Commit: c0e1f99, github.com/apache/spark/pull/1886 + + [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences + Michael Armbrust + 2014-08-26 16:29:14 -0700 + Commit: a308a16, github.com/apache/spark/pull/2109 + + [SPARK-2839][MLlib] Stats Toolkit documentation updated + Burak + 2014-08-26 15:18:42 -0700 + Commit: 2715eb7, github.com/apache/spark/pull/2130 + + [SPARK-3226][MLLIB] doc update for native libraries + Xiangrui Meng + 2014-08-26 15:12:27 -0700 + Commit: 5ff9000, github.com/apache/spark/pull/2128 + + [SPARK-3063][SQL] ExistingRdd should convert Map to catalyst Map. + Takuya UESHIN + 2014-08-26 15:04:08 -0700 + Commit: 5d981a4, github.com/apache/spark/pull/1963 + + [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull. + Takuya UESHIN + 2014-08-26 13:22:55 -0700 + Commit: 35a5853, github.com/apache/spark/pull/1889 + + [SPARK-2871] [PySpark] add histgram() API + Davies Liu + 2014-08-26 13:04:30 -0700 + Commit: 83d2730, github.com/apache/spark/pull/2091 + + [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext + chutium + 2014-08-26 11:51:26 -0700 + Commit: 3a9d874, github.com/apache/spark/pull/2039 + + [SPARK-2886] Use more specific actor system name than "spark" + Andrew Or + 2014-08-25 23:36:09 -0700 + Commit: 0f947f1, github.com/apache/spark/pull/1810 + + [Spark-3222] [SQL] Cross join support in HiveQL + Daoyuan Wang , adrian-wang + 2014-08-25 22:56:35 -0700 + Commit: 48a0749, github.com/apache/spark/pull/2124 + + SPARK-2481: The environment variables SPARK_HISTORY_OPTS is covered in spark-env.sh + witgo , GuoQiang Li + 2014-08-25 19:22:27 -0700 + Commit: 4d6a0e9, github.com/apache/spark/pull/1341 + + [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile + Chia-Yung Su + 2014-08-25 18:20:19 -0700 + Commit: b5dc9b4, github.com/apache/spark/pull/1959 + + [SQL] logWarning should be logInfo in getResultSetSchema + wangfei + 2014-08-25 17:46:43 -0700 + Commit: 957b356, github.com/apache/spark/pull/1939 + + [SPARK-3058] [SQL] Support EXTENDED for EXPLAIN + Cheng Hao + 2014-08-25 17:43:56 -0700 + Commit: f8ac8ed, github.com/apache/spark/pull/1962 + + [SPARK-2929][SQL] Refactored Thrift server and CLI suites + Cheng Lian + 2014-08-25 16:29:59 -0700 + Commit: 292f28d, github.com/apache/spark/pull/1856 + + [SPARK-3204][SQL] MaxOf would be foldable if both left and right are foldable. + Takuya UESHIN + 2014-08-25 16:27:00 -0700 + Commit: 19b01d6, github.com/apache/spark/pull/2116 + + Fixed a typo in docs/running-on-mesos.md + Cheng Lian + 2014-08-25 14:56:51 -0700 + Commit: 8d33a6d, github.com/apache/spark/pull/2119 + + [FIX] fix error message in sendMessageReliably + Xiangrui Meng + 2014-08-25 14:55:20 -0700 + Commit: d892062, github.com/apache/spark/pull/2120 + + SPARK-2798 [BUILD] Correct several small errors in Flume module pom.xml files + Sean Owen + 2014-08-25 13:29:07 -0700 + Commit: ff616fd, github.com/apache/spark/pull/1726 + + [SPARK-2495][MLLIB] make KMeans constructor public + Xiangrui Meng + 2014-08-25 12:30:02 -0700 + Commit: 69a17f1, github.com/apache/spark/pull/2112 + + [SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId() + Davies Liu + 2014-08-24 21:16:05 -0700 + Commit: b82da3d, github.com/apache/spark/pull/2092 + + [MLlib][SPARK-2997] Update SVD documentation to reflect roughly square + Reza Zadeh + 2014-08-24 17:35:54 -0700 + Commit: 749bddc, github.com/apache/spark/pull/2070 + + [SPARK-2841][MLlib] Documentation for feature transformations + DB Tsai + 2014-08-24 17:33:33 -0700 + Commit: a4db81a, github.com/apache/spark/pull/2068 + + [SPARK-3192] Some scripts have 2 space indentation but other scripts have 4 space indentation. + Kousuke Saruta + 2014-08-24 09:43:44 -0700 + Commit: ce14cd1, github.com/apache/spark/pull/2104 + + [SPARK-2967][SQL] Follow-up: Also copy hash expressions in sort based shuffle fix. + Michael Armbrust + 2014-08-23 16:21:08 -0700 + Commit: e23f0bc, github.com/apache/spark/pull/2072 + + [SPARK-2554][SQL] CountDistinct partial aggregation and object allocation improvements + Michael Armbrust , Gregory Owen + 2014-08-23 16:19:10 -0700 + Commit: 7112da8, github.com/apache/spark/pull/1935 + + [SQL] Make functionRegistry in HiveContext transient. + Yin Huai + 2014-08-23 12:46:41 -0700 + Commit: 9309786, github.com/apache/spark/pull/2074 + + [SPARK-2963] REGRESSION - The description about how to build for using CLI and Thrift JDBC server is absent in proper document - + Kousuke Saruta + 2014-08-22 22:28:05 -0700 + Commit: 5689660, github.com/apache/spark/pull/2080 + + [SPARK-3169] Removed dependency on spark streaming test from spark flume sink + Tathagata Das + 2014-08-22 21:34:48 -0700 + Commit: cd73631, github.com/apache/spark/pull/2101 + + Revert "HOTFIX:Temporarily removing flume sink test in 1.1 branch" + Patrick Wendell + 2014-08-22 21:31:52 -0700 + Commit: 385c4f2 + + [SPARK-2840] [mllib] DecisionTree doc update (Java, Python examples) + Joseph K. Bradley + 2014-08-21 00:17:29 -0700 + Commit: 1e5d9cb, github.com/apache/spark/pull/2063 + + BUILD: Bump Hadoop versions in the release build. + Patrick Wendell + 2014-08-20 12:18:41 -0700 + Commit: da0a701 + + HOTFIX:Temporarily removing flume sink test in 1.1 branch + Patrick Wendell + 2014-08-20 22:24:22 -0700 + Commit: 1d5e84a + + [HOTFIX][STREAMING] Allow the JVM/Netty to decide which port to bind to in Flume Polling Tests. + Hari Shreedharan + 2014-08-17 19:50:31 -0700 + Commit: 4485665, github.com/apache/spark/pull/1820 + + [HOTFIX][Streaming] Handle port collisions in flume polling test + Andrew Or + 2014-08-06 16:34:53 -0700 + Commit: 3f91e9d, github.com/apache/spark/pull/1803 + + [SPARK-2843][MLLIB] add a section about regularization parameter in ALS + Xiangrui Meng + 2014-08-20 17:47:39 -0700 + Commit: eba399b, github.com/apache/spark/pull/2064 + + [SPARK-3143][MLLIB] add tf-idf user guide + Xiangrui Meng + 2014-08-20 17:41:36 -0700 + Commit: 1af68ca, github.com/apache/spark/pull/2061 + + [SPARK-3140] Clarify confusing PySpark exception message + Andrew Or + 2014-08-20 17:07:39 -0700 + Commit: f8bcb12, github.com/apache/spark/pull/2067 + + [SPARK-2298] Encode stage attempt in SparkListener & UI. + Reynold Xin + 2014-08-20 15:37:27 -0700 + Commit: dc05282, github.com/apache/spark/pull/1545 + + [SPARK-2169] Don't copy appName / basePath everywhere. + Marcelo Vanzin + 2014-08-18 13:25:30 -0700 + Commit: 2c1683e, github.com/apache/spark/pull/1252 + + [SPARK-2846][SQL] Add configureInputJobPropertiesForStorageHandler to initialization of job conf + Alex Liu + 2014-08-20 16:14:06 -0700 + Commit: 64e136a, github.com/apache/spark/pull/1927 + + SPARK_LOGFILE and SPARK_ROOT_LOGGER no longer need in spark-daemon.sh + wangfei + 2014-08-20 16:00:46 -0700 + Commit: 5f72d7b, github.com/apache/spark/pull/2057 + + [SPARK-2967][SQL] Fix sort based shuffle for spark sql. + Michael Armbrust + 2014-08-20 15:51:14 -0700 + Commit: 311831d, github.com/apache/spark/pull/2066 + + [SPARK-2849] Handle driver configs separately in client mode + Andrew Or + 2014-08-20 15:01:47 -0700 + Commit: beb705a, github.com/apache/spark/pull/1845 + + [SPARK-3149] Connection establishment information is not enough. + Kousuke Saruta + 2014-08-20 14:04:39 -0700 + Commit: 25b01fd, github.com/apache/spark/pull/2060 + + [SPARK-3062] [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled + Kousuke Saruta + 2014-08-20 13:26:11 -0700 + Commit: 5095851, github.com/apache/spark/pull/1970 + + [SPARK-3126][SPARK-3127][SQL] Fixed HiveThriftServer2Suite + Cheng Lian + 2014-08-20 12:57:39 -0700 + Commit: 99ca704, github.com/apache/spark/pull/2036 + + SPARK-3092 [SQL]: Always include the thriftserver when -Phive is enabled. + Patrick Wendell + 2014-08-20 12:13:31 -0700 + Commit: ca7322d, github.com/apache/spark/pull/2006 + + [SPARK-3054][STREAMING] Add unit tests for Spark Sink. + Hari Shreedharan , Hari Shreedharan + 2014-08-20 04:09:54 -0700 + Commit: 9b29099, github.com/apache/spark/pull/1958 + + [SPARK-3141] [PySpark] fix sortByKey() with take() + Davies Liu + 2014-08-19 22:43:49 -0700 + Commit: 5b22ebf, github.com/apache/spark/pull/2045 + + [DOCS] Fixed wrong links + Ken Takagiwa + 2014-08-19 22:43:22 -0700 + Commit: f8c908e, github.com/apache/spark/pull/2042 + + [SPARK-2974] [SPARK-2975] Fix two bugs related to spark.local.dirs + Josh Rosen + 2014-08-19 22:42:50 -0700 + Commit: 5d1a878, github.com/apache/spark/pull/2002 + + [SPARK-3142][MLLIB] output shuffle data directly in Word2Vec + Xiangrui Meng + 2014-08-19 22:16:22 -0700 + Commit: a5bc9c6, github.com/apache/spark/pull/2049 + + [SPARK-3119] Re-implementation of TorrentBroadcast. + Reynold Xin + 2014-08-19 22:11:13 -0700 + Commit: 08c9973, github.com/apache/spark/pull/2030 + + [HOTFIX][Streaming][MLlib] use temp folder for checkpoint + Xiangrui Meng + 2014-08-19 22:05:29 -0700 + Commit: d5db95b, github.com/apache/spark/pull/2046 + + [SPARK-3130][MLLIB] detect negative values in naive Bayes + Xiangrui Meng + 2014-08-19 21:01:23 -0700 + Commit: 148e45b, github.com/apache/spark/pull/2038 + + [SQL] add note of use synchronizedMap in SQLConf + wangfei , scwf + 2014-08-19 19:37:02 -0700 + Commit: 607735c, github.com/apache/spark/pull/1996 + + [SPARK-3112][MLLIB] Add documentation and example for StreamingLR + freeman + 2014-08-19 18:07:42 -0700 + Commit: d75464d, github.com/apache/spark/pull/2047 + + [MLLIB] minor update to word2vec + Xiangrui Meng + 2014-08-19 17:41:37 -0700 + Commit: 023ed7c, github.com/apache/spark/pull/2043 + + [SPARK-2468] Netty based block server / client module + Reynold Xin + 2014-08-19 17:40:35 -0700 + Commit: 66b4c81, github.com/apache/spark/pull/1971 + + [SPARK-3136][MLLIB] Create Java-friendly methods in RandomRDDs + Xiangrui Meng + 2014-08-19 16:06:48 -0700 + Commit: d371c71, github.com/apache/spark/pull/2041 + + [SPARK-2790] [PySpark] fix zip with serializers which have different batch sizes. + Davies Liu + 2014-08-19 14:46:32 -0700 + Commit: 3540d4b, github.com/apache/spark/pull/1894 + + Move a bracket in validateSettings of SparkConf + hzw19900416 + 2014-08-19 14:04:49 -0700 + Commit: f6b4ab8, github.com/apache/spark/pull/2012 + + SPARK-2333 - spark_ec2 script should allow option for existing security group + Vida Ha + 2014-08-19 13:35:05 -0700 + Commit: c3952b0, github.com/apache/spark/pull/1899 + + [SPARK-3128][MLLIB] Use streaming test suite for StreamingLR + freeman + 2014-08-19 13:28:57 -0700 + Commit: 04a3208, github.com/apache/spark/pull/2037 + + [SPARK-3089] Fix meaningless error message in ConnectionManager + Kousuke Saruta + 2014-08-19 10:15:11 -0700 + Commit: 5d895ad, github.com/apache/spark/pull/2000 + + [SPARK-3072] YARN - Exit when reach max number failed executors + Thomas Graves + 2014-08-19 09:40:31 -0500 + Commit: 1418893, github.com/apache/spark/pull/2022 + + Fix typo in decision tree docs + Matt Forbes + 2014-08-18 21:43:32 -0700 + Commit: f3b0f34, github.com/apache/spark/pull/1837 + + [SPARK-3116] Remove the excessive lockings in TorrentBroadcast + Reynold Xin + 2014-08-18 20:51:41 -0700 + Commit: b6d8e66, github.com/apache/spark/pull/2028 + + [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL. + Josh Rosen , Davies Liu + 2014-08-18 20:42:19 -0700 + Commit: 3a03259, github.com/apache/spark/pull/2026. + + [SPARK-3108][MLLIB] add predictOnValues to StreamingLR and fix predictOn + Xiangrui Meng + 2014-08-18 18:20:54 -0700 + Commit: 7d069bf, github.com/apache/spark/pull/2023 + + [SPARK-2850] [SPARK-2626] [mllib] MLlib stats examples + small fixes + Joseph K. Bradley + 2014-08-18 18:01:39 -0700 + Commit: e3f89e9, github.com/apache/spark/pull/1878 + + [mllib] DecisionTree: treeAggregate + Python example bug fix + Joseph K. Bradley + 2014-08-18 14:40:05 -0700 + Commit: 98778ff, github.com/apache/spark/pull/2015 + + [SPARK-2718] [yarn] Handle quotes and other characters in user args. + Marcelo Vanzin + 2014-08-18 14:10:10 -0700 + Commit: 25cabd7, github.com/apache/spark/pull/1724 + + [SPARK-3103] [PySpark] fix saveAsTextFile() with utf-8 + Davies Liu + 2014-08-18 13:58:35 -0700 + Commit: e083334, github.com/apache/spark/pull/2018 + + [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables. + Michael Armbrust , Yin Huai + 2014-08-18 13:17:10 -0700 + Commit: cc4015d, github.com/apache/spark/pull/1819 + + [SPARK-3091] [SQL] Add support for caching metadata on Parquet files + Matei Zaharia + 2014-08-18 11:00:10 -0700 + Commit: 2ae2857, github.com/apache/spark/pull/2005 + + SPARK-3025 [SQL]: Allow JDBC clients to set a fair scheduler pool + Patrick Wendell + 2014-08-18 10:52:20 -0700 + Commit: 496f62d, github.com/apache/spark/pull/1937 + + [SPARK-3085] [SQL] Use compact data structures in SQL joins + Matei Zaharia + 2014-08-18 10:45:24 -0700 + Commit: 4da76fc, github.com/apache/spark/pull/1993 + + [SPARK-3084] [SQL] Collect broadcasted tables in parallel in joins + Matei Zaharia + 2014-08-18 10:05:52 -0700 + Commit: 55e9dd6, github.com/apache/spark/pull/1990 + + SPARK-3096: Include parquet hive serde by default in build + Patrick Wendell + 2014-08-18 10:00:46 -0700 + Commit: ec0b91e, github.com/apache/spark/pull/2009 + + [SPARK-2862] histogram method fails on some choices of bucketCount + Chandan Kumar + 2014-08-18 09:52:25 -0700 + Commit: 12f16ba, github.com/apache/spark/pull/1787 + + [MLlib] Remove transform(dataset: RDD[String]) from Word2Vec public API + Liquan Pei + 2014-08-18 01:15:45 -0700 + Commit: e0bc333, github.com/apache/spark/pull/2010 + + [SPARK-2842][MLlib]Word2Vec documentation + Liquan Pei + 2014-08-17 23:30:47 -0700 + Commit: 518258f, github.com/apache/spark/pull/2003 + + [SPARK-3097][MLlib] Word2Vec performance improvement + Liquan Pei + 2014-08-17 23:29:44 -0700 + Commit: 708cde9, github.com/apache/spark/pull/1932 + + SPARK-2900. aggregate inputBytes per stage + Sandy Ryza + 2014-08-17 22:39:06 -0700 + Commit: 0506539, github.com/apache/spark/pull/1826 + + SPARK-2884: Create binary builds in parallel with release script. + Patrick Wendell + 2014-08-17 22:29:58 -0700 + Commit: a5ae720 + + [SPARK-3087][MLLIB] fix col indexing bug in chi-square and add a check for number of distinct values + Xiangrui Meng + 2014-08-17 20:53:18 -0700 + Commit: 8438daf, github.com/apache/spark/pull/1997 + + [SPARK-1981] updated streaming-kinesis.md + Chris Fregly + 2014-08-17 19:33:15 -0700 + Commit: 8263567, github.com/apache/spark/pull/1757 + + [SQL] Improve debug logging and toStrings. + Michael Armbrust + 2014-08-17 19:00:38 -0700 + Commit: 4f776df, github.com/apache/spark/pull/2004 + + Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" + Michael Armbrust + 2014-08-17 18:10:45 -0700 + Commit: c6a0091, github.com/apache/spark/pull/2007 + + SPARK-2881: Upgrade to Snappy 1.0.5.3 to avoid SPARK-2881. + Patrick Wendell + 2014-08-17 15:48:39 -0700 + Commit: d411f41, github.com/apache/spark/pull/1999 + + [SPARK-3042] [mllib] DecisionTree Filter top-down instead of bottom-up + Joseph K. Bradley + 2014-08-16 23:53:14 -0700 + Commit: 91af120, github.com/apache/spark/pull/1975 + + [SPARK-3077][MLLIB] fix some chisq-test + Xiangrui Meng + 2014-08-16 21:16:27 -0700 + Commit: 413a329, github.com/apache/spark/pull/1982 + + In the stop method of ConnectionManager to cancel the ackTimeoutMonitor + GuoQiang Li + 2014-08-16 20:05:55 -0700 + Commit: f02e327, github.com/apache/spark/pull/1989 + + [SPARK-1065] [PySpark] improve supporting for large broadcast + Davies Liu + 2014-08-16 16:59:34 -0700 + Commit: 5dd571c, github.com/apache/spark/pull/1912 + + [SPARK-3035] Wrong example with SparkContext.addFile + iAmGhost + 2014-08-16 16:48:38 -0700 + Commit: 721f2fd, github.com/apache/spark/pull/1942 + + [SPARK-3081][MLLIB] rename RandomRDDGenerators to RandomRDDs + Xiangrui Meng + 2014-08-16 15:14:43 -0700 + Commit: a12d3ae, github.com/apache/spark/pull/1979 + + [SPARK-3048][MLLIB] add LabeledPoint.parse and remove loadStreamingLabeledPoints + Xiangrui Meng + 2014-08-16 15:13:34 -0700 + Commit: 0b354be, github.com/apache/spark/pull/1952 + + [SPARK-2677] BasicBlockFetchIterator#next can wait forever + Kousuke Saruta + 2014-08-16 14:15:58 -0700 + Commit: bd3ce2f, github.com/apache/spark/pull/1632 + + [SQL] Using safe floating-point numbers in doctest + Cheng Lian + 2014-08-16 11:26:51 -0700 + Commit: 8c79574, github.com/apache/spark/pull/1925 + + [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager + Josh Rosen + 2014-08-16 00:04:55 -0700 + Commit: 0e0ec2e, github.com/apache/spark/pull/1976 + + [SPARK-3045] Make Serializer interface Java friendly + Reynold Xin + 2014-08-15 23:12:34 -0700 + Commit: fcf30cd, github.com/apache/spark/pull/1948 + + [SPARK-3015] Block on cleaning tasks to prevent Akka timeouts + Andrew Or + 2014-08-15 22:55:32 -0700 + Commit: 2541537, github.com/apache/spark/pull/1931 + + [SPARK-3001][MLLIB] Improve Spearman's correlation + Xiangrui Meng + 2014-08-15 21:07:55 -0700 + Commit: ce06d7f, github.com/apache/spark/pull/1917 + + [SPARK-3078][MLLIB] Make LRWithLBFGS API consistent with others + Xiangrui Meng + 2014-08-15 21:04:29 -0700 + Commit: c085011, github.com/apache/spark/pull/1973 + + [SPARK-3046] use executor's class loader as the default serializer classloader + Reynold Xin + 2014-08-15 17:04:15 -0700 + Commit: 077213b, github.com/apache/spark/pull/1972 + + [SPARK-3022] [SPARK-3041] [mllib] Call findBins once per level + unordered feature bug fix + Joseph K. Bradley + 2014-08-15 14:50:10 -0700 + Commit: 407ea9f, github.com/apache/spark/pull/1950 + + SPARK-3028. sparkEventToJson should support SparkListenerExecutorMetrics... + Sandy Ryza + 2014-08-15 11:35:08 -0700 + Commit: 63376a0, github.com/apache/spark/pull/1961 + + Revert "[SPARK-2468] Netty based block server / client module" + Patrick Wendell + 2014-08-15 09:01:35 -0700 + Commit: b066af4 + + [SPARK-2924] remove default args to overloaded methods + Anand Avati + 2014-08-15 08:53:52 -0700 + Commit: debb3e3, github.com/apache/spark/pull/1704 + + [SPARK-2468] Netty based block server / client module + Reynold Xin + 2014-08-14 19:01:33 -0700 + Commit: 3f23d2a, github.com/apache/spark/pull/1907 + + [SPARK-2936] Migrate Netty network module from Java to Scala + Reynold Xin + 2014-08-10 20:36:54 -0700 + Commit: d3cce58, github.com/apache/spark/pull/1865 + + [SPARK-2736] PySpark converter and example script for reading Avro files + Kan Zhang + 2014-08-14 19:03:51 -0700 + Commit: 72e730e, github.com/apache/spark/pull/1916 + + [SPARK-3027] TaskContext: tighten visibility and provide Java friendly callback API + Reynold Xin + 2014-08-14 18:37:02 -0700 + Commit: f99e4fc, github.com/apache/spark/pull/1938 + + Make dev/mima runnable on Mac OS X. + Reynold Xin + 2014-08-14 16:27:11 -0700 + Commit: 475a35b, github.com/apache/spark/pull/1953 + + SPARK-3009: Reverted readObject method in ApplicationInfo so that Applic... + Jacek Lewandowski + 2014-08-14 15:01:39 -0700 + Commit: f5d9176, github.com/apache/spark/pull/1947 + + Revert [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile + Michael Armbrust + 2014-08-14 13:00:21 -0700 + Commit: c39a3f3, github.com/apache/spark/pull/1949 + + [SPARK-2979][MLlib] Improve the convergence rate by minimizing the condition number + DB Tsai + 2014-08-14 11:56:13 -0700 + Commit: dc8ef93, github.com/apache/spark/pull/1897 + + Minor cleanup of metrics.Source + Reynold Xin + 2014-08-14 11:22:41 -0700 + Commit: a3dc54f, github.com/apache/spark/pull/1943 + + [SPARK-2925] [sql]fix spark-sql and start-thriftserver shell bugs when set --driver-java-options + wangfei , wangfei + 2014-08-14 10:55:51 -0700 + Commit: df25acd, github.com/apache/spark/pull/1851 + + [SQL] Python JsonRDD UTF8 Encoding Fix + Ahir Reddy + 2014-08-14 10:48:52 -0700 + Commit: 850abaa, github.com/apache/spark/pull/1914 + + [SPARK-2927][SQL] Add a conf to configure if we always read Binary columns stored in Parquet as String columns + Yin Huai + 2014-08-14 10:46:33 -0700 + Commit: de501e1, github.com/apache/spark/pull/1855 + + [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile + Chia-Yung Su + 2014-08-14 10:43:08 -0700 + Commit: 221c84e, github.com/apache/spark/pull/1924 + + SPARK-2893: Do not swallow Exceptions when running a custom kryo registrator + Graham Dennis + 2014-08-14 02:24:18 -0700 + Commit: af809de, github.com/apache/spark/pull/1827 + + [SPARK-3029] Disable local execution of Spark jobs by default + Aaron Davidson + 2014-08-14 01:37:38 -0700 + Commit: 0cb2b82, github.com/apache/spark/pull/1321 + + [SPARK-2995][MLLIB] add ALS.setIntermediateRDDStorageLevel + Xiangrui Meng + 2014-08-13 23:53:44 -0700 + Commit: 1baf06f, github.com/apache/spark/pull/1913 + + [Docs] Add missing tags (minor) + Andrew Or + 2014-08-13 23:24:23 -0700 + Commit: bf7c6e1, github.com/apache/spark/pull/1936 + + [SPARK-3006] Failed to execute spark-shell in Windows OS + Masayoshi TSUZUKI + 2014-08-13 22:17:07 -0700 + Commit: dcd99c3, github.com/apache/spark/pull/1918 + + SPARK-3020: Print completed indices rather than tasks in web UI + Patrick Wendell + 2014-08-13 18:08:38 -0700 + Commit: c6cb55a, github.com/apache/spark/pull/1933 + + [SPARK-2986] [SQL] fixed: setting properties does not effect + guowei + 2014-08-13 17:45:24 -0700 + Commit: a8d2649, github.com/apache/spark/pull/1904 + + [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled + Kousuke Saruta + 2014-08-13 17:42:38 -0700 + Commit: b5b632c, github.com/apache/spark/pull/1891 + + [SPARK-2935][SQL]Fix parquet predicate push down bug + Michael Armbrust + 2014-08-13 17:40:59 -0700 + Commit: e8e7f17, github.com/apache/spark/pull/1863 + + [SPARK-2650][SQL] More precise initial buffer size estimation for in-memory column buffer + Cheng Lian + 2014-08-13 17:37:55 -0700 + Commit: ee7d2cc, github.com/apache/spark/pull/1901 + + [SPARK-2994][SQL] Support for udfs that take complex types + Michael Armbrust + 2014-08-13 17:35:38 -0700 + Commit: 71b8408, github.com/apache/spark/pull/1915 + + [SPARK-2817] [SQL] add "show create table" support + tianyi , tianyi , tianyi + 2014-08-13 16:50:02 -0700 + Commit: 0fb1198, github.com/apache/spark/pull/1760 + + [SPARK-3004][SQL] Added null checking when retrieving row set + Cheng Lian + 2014-08-13 16:27:50 -0700 + Commit: 8732375, github.com/apache/spark/pull/1920 + + [MLLIB] use Iterator.fill instead of Array.fill + Xiangrui Meng + 2014-08-13 16:20:49 -0700 + Commit: e63bf87, github.com/apache/spark/pull/1930 + + [SPARK-2983] [PySpark] improve performance of sortByKey() + Davies Liu + 2014-08-13 14:57:12 -0700 + Commit: a7bc21c, github.com/apache/spark/pull/1898 + + [SPARK-3013] [SQL] [PySpark] convert array into list + Davies Liu + 2014-08-13 14:56:11 -0700 + Commit: 9936020, github.com/apache/spark/pull/1928 + + [SPARK-2963] [SQL] There no documentation about building to use HiveServer and CLI for SparkSQL + Kousuke Saruta + 2014-08-13 14:42:57 -0700 + Commit: 78f2f99, github.com/apache/spark/pull/1885 + + [SPARK-2993] [MLLib] colStats (wrapper around MultivariateStatisticalSummary) in Statistics + Doris Xin + 2014-08-12 23:47:42 -0700 + Commit: 5ebeb3f, github.com/apache/spark/pull/1911 + + [SPARK-1777 (partial)] bugfix: make size of requested memory correctly + Zhang, Liye + 2014-08-12 23:43:36 -0700 + Commit: ec5e2b0, github.com/apache/spark/pull/1892 + + Use transferTo when copy merge files in ExternalSorter + Raymond Liu + 2014-08-12 23:19:35 -0700 + Commit: be674b3, github.com/apache/spark/pull/1884 + + [SPARK-2953] Allow using short names for io compression codecs + Reynold Xin + 2014-08-12 22:50:29 -0700 + Commit: 837bf60, github.com/apache/spark/pull/1873 + + SPARK-2830 [MLlib]: re-organize mllib documentation + Ameet Talwalkar + 2014-08-12 17:15:21 -0700 + Commit: cffd9bb, github.com/apache/spark/pull/1908 + + fix flaky tests + Davies Liu + 2014-08-12 16:26:01 -0700 + Commit: b5f8083, github.com/apache/spark/pull/1910 + + [MLlib] Correctly set vectorSize and alpha + Liquan Pei + 2014-08-12 00:28:00 -0700 + Commit: 2a8117a, github.com/apache/spark/pull/1900 + + [SPARK-2923][MLLIB] Implement some basic BLAS routines + Xiangrui Meng + 2014-08-11 22:33:45 -0700 + Commit: 872c170, github.com/apache/spark/pull/1849 + + [SQL] [SPARK-2826] Reduce the memory copy while building the hashmap for HashOuterJoin + Cheng Hao + 2014-08-11 20:45:14 -0700 + Commit: f66f260, github.com/apache/spark/pull/1765 + + [SPARK-2650][SQL] Build column buffers in smaller batches + Michael Armbrust + 2014-08-11 20:21:56 -0700 + Commit: 779d1eb, github.com/apache/spark/pull/1880 + + [SPARK-2968][SQL] Fix nullabilities of Explode. + Takuya UESHIN + 2014-08-11 20:18:03 -0700 + Commit: 54b387f, github.com/apache/spark/pull/1888 + + [SPARK-2965][SQL] Fix HashOuterJoin output nullabilities. + Takuya UESHIN + 2014-08-11 20:15:01 -0700 + Commit: dcbf079, github.com/apache/spark/pull/1887 + + [SQL] A tiny refactoring in HiveContext#analyze + Yin Huai + 2014-08-11 20:11:29 -0700 + Commit: fd8173f, github.com/apache/spark/pull/1881 + + [sql]use SparkSQLEnv.stop() in ShutdownHook + wangfei + 2014-08-11 20:10:13 -0700 + Commit: 6d0af52, github.com/apache/spark/pull/1852 + + [SPARK-2590][SQL] Added option to handle incremental collection, disabled by default + Cheng Lian + 2014-08-11 20:08:06 -0700 + Commit: cf2f807, github.com/apache/spark/pull/1853 + + [SPARK-2844][SQL] Correctly set JVM HiveContext if it is passed into Python HiveContext constructor + Ahir Reddy + 2014-08-11 20:06:06 -0700 + Commit: 8cb4e5b, github.com/apache/spark/pull/1768 + + [SPARK-2934][MLlib] Adding LogisticRegressionWithLBFGS Interface + DB Tsai + 2014-08-11 19:49:29 -0700 + Commit: 8f6e2e9, github.com/apache/spark/pull/1862 + + [SPARK-2515][mllib] Chi Squared test + Doris Xin + 2014-08-11 19:22:14 -0700 + Commit: 7e31f7c, github.com/apache/spark/pull/1733 + + [SPARK-2931] In TaskSetManager, reset currentLocalityIndex after recomputing locality levels + Josh Rosen + 2014-08-11 19:15:01 -0700 + Commit: 6c64d57, github.com/apache/spark/pull/1896 + + [SPARK-2952] Enable logging actor messages at DEBUG level + Reynold Xin + 2014-08-11 15:25:21 -0700 + Commit: 6ec1374, github.com/apache/spark/pull/1870 + + [PySpark] [SPARK-2954] [SPARK-2948] [SPARK-2910] [SPARK-2101] Python 2.6 Fixes + Josh Rosen + 2014-08-11 11:54:09 -0700 + Commit: 09b8a3c, github.com/apache/spark/pull/1868. + + [SPARK-2937] Separate out samplyByKeyExact as its own API in PairRDDFunction + Doris Xin , Xiangrui Meng + 2014-08-10 16:31:07 -0700 + Commit: 3def842, github.com/apache/spark/pull/1866 + + [SPARK-2898] [PySpark] fix bugs in deamon.py + Davies Liu + 2014-08-10 13:00:38 -0700 + Commit: 92daffe, github.com/apache/spark/pull/1842 + + Remove extra semicolon in Task.scala + GuoQiang Li + 2014-08-10 12:12:22 -0700 + Commit: bb23b11, github.com/apache/spark/pull/1876 + + Turn UpdateBlockInfo into case class. + Reynold Xin + 2014-08-09 23:06:54 -0700 + Commit: 076ddda, github.com/apache/spark/pull/1872 + + Updated Spark SQL README to include the hive-thriftserver module + Reynold Xin + 2014-08-09 22:05:36 -0700 + Commit: e8f8e5f, github.com/apache/spark/pull/1867 + + [SPARK-2894] spark-shell doesn't accept flags + Kousuke Saruta , Cheng Lian + 2014-08-09 21:10:43 -0700 + Commit: ba223b8, github.com/apache/spark/pull/1715, + + [SPARK-1766] sorted functions to meet pedantic requirements + Chris Cope + 2014-08-09 20:58:56 -0700 + Commit: 4a7f3ef, github.com/apache/spark/pull/1859 + + [SPARK-2861] Fix Doc comment of histogram method + Chandan Kumar + 2014-08-09 00:45:54 -0700 + Commit: 71fcd2e, github.com/apache/spark/pull/1786 + + [SPARK-2635] Fix race condition at SchedulerBackend.isReady in standalone mode + li-zhihui , Li Zhihui + 2014-08-08 22:52:56 -0700 + Commit: 3311da2, github.com/apache/spark/pull/1525 + + [SPARK-2897][SPARK-2920]TorrentBroadcast does use the serializer class specified in the spark option "spark.serializer" + GuoQiang Li + 2014-08-08 16:57:26 -0700 + Commit: dd11e4e, github.com/apache/spark/pull/1836 + + [SPARK-1997][MLLIB] update breeze to 0.9 + Xiangrui Meng + 2014-08-08 15:07:31 -0700 + Commit: 8fba6de, github.com/apache/spark/pull/1749 + + [SPARK-2700] [SQL] Hidden files (such as .impala_insert_staging) should be filtered out by sqlContext.parquetFile + chutium + 2014-08-08 13:31:08 -0700 + Commit: e264503, github.com/apache/spark/pull/1691 + + [SPARK-2919] [SQL] Basic support for analyze command in HiveQl + Yin Huai + 2014-08-08 11:23:58 -0700 + Commit: daa090f, github.com/apache/spark/pull/1848 + + [SPARK-2877] [SQL] MetastoreRelation should use SparkClassLoader when creating the tableDesc + Yin Huai + 2014-08-08 11:15:16 -0700 + Commit: 8b0188b, github.com/apache/spark/pull/1806 + + [SPARK-2908] [SQL] JsonRDD.nullTypeToStringType does not convert all NullType to StringType + Yin Huai + 2014-08-08 11:10:11 -0700 + Commit: 544a909, github.com/apache/spark/pull/1840 + + [SPARK-2888] [SQL] Fix addColumnMetadataToConf in HiveTableScan + Yin Huai + 2014-08-08 11:01:51 -0700 + Commit: 3eb5dd0, github.com/apache/spark/pull/1817 + + [SPARK-2904] Remove non-used local variable in SparkSubmitArguments + Kousuke Saruta + 2014-08-07 18:53:15 -0700 + Commit: a54b5d9, github.com/apache/spark/pull/1834 + + SPARK-2565. Update ShuffleReadMetrics as blocks are fetched + Sandy Ryza + 2014-08-07 18:09:03 -0700 + Commit: aab7735, github.com/apache/spark/pull/1507 + + SPARK-2787: Make sort-based shuffle write files directly when there's no sorting/aggregation and # partitions is small + Matei Zaharia + 2014-08-07 18:04:49 -0700 + Commit: 0f2274f, github.com/apache/spark/pull/1799 + + SPARK-2899 Doc generation is back to working in new SBT Build. + Prashant Sharma + 2014-08-07 16:24:22 -0700 + Commit: 30369b8, github.com/apache/spark/pull/1830 + + SPARK-2905 Fixed path sbin => bin + Oleg Danilov + 2014-08-07 15:48:44 -0700 + Commit: c65c810, github.com/apache/spark/pull/1835 + + [SPARK-2852][MLLIB] Separate model from IDF/StandardScaler algorithms + Xiangrui Meng + 2014-08-07 11:28:12 -0700 + Commit: f705c1d, github.com/apache/spark/pull/1814 + + [mllib] DecisionTree Strategy parameter checks + Joseph K. Bradley + 2014-08-07 00:20:38 -0700 + Commit: c089429, github.com/apache/spark/pull/1821 + + SPARK-2879 part 2 [BUILD] Use HTTPS to access Maven Central and other repos + Sean Owen + 2014-08-07 00:04:18 -0700 + Commit: d6cd6fd0, github.com/apache/spark/pull/1828 + + [SPARK-2851] [mllib] DecisionTree Python consistency update + Joseph K. Bradley + 2014-08-06 22:58:59 -0700 + Commit: c9f0944, github.com/apache/spark/pull/1798 + + [SPARK-2887] fix bug of countApproxDistinct() when have more than one partition + Davies Liu + 2014-08-06 21:22:13 -0700 + Commit: cc8a7e9, github.com/apache/spark/pull/1812 + + Updating versions for Spark 1.1.0 + Patrick Wendell + 2014-08-06 19:11:39 -0700 + Commit: cf35b56 + + HOTFIX: Support custom Java 7 location + Patrick Wendell + 2014-08-06 18:45:03 -0700 + Commit: 53fa048 + + SPARK-2879 [BUILD] Use HTTPS to access Maven Central and other repos + Sean Owen + 2014-08-06 18:13:35 -0700 + Commit: 40284a9, github.com/apache/spark/pull/1805 + + [SPARK-2583] ConnectionManager error reporting + Kousuke Saruta , Josh Rosen + 2014-08-06 17:27:55 -0700 + Commit: 3f92ce4, github.com/apache/spark/pull/1758 + + SPARK-2882: Spark build now checks local maven cache for dependencies + Gregory Owen + 2014-08-06 16:52:00 -0700 + Commit: c2ae0b0, github.com/apache/spark/pull/1818 + + [PySpark] Add blanklines to Python docstrings so example code renders correctly + RJ Nowling + 2014-08-06 14:12:21 -0700 + Commit: a314e29, github.com/apache/spark/pull/1808 + + [SPARK-2852][MLLIB] API consistency for `mllib.feature` + Xiangrui Meng + 2014-08-06 14:07:51 -0700 + Commit: e654cfd, github.com/apache/spark/pull/1807 + + SPARK-2566. Update ShuffleWriteMetrics incrementally + Sandy Ryza + 2014-08-06 13:10:33 -0700 + Commit: a65c9ac, github.com/apache/spark/pull/1481 + + [SPARK-2627] [PySpark] have the build enforce PEP 8 automatically + Nicholas Chammas , nchammas + 2014-08-06 12:58:24 -0700 + Commit: 4c19614, github.com/apache/spark/pull/1744 + + [SPARK-2678][Core][SQL] A workaround for SPARK-2678 + Cheng Lian + 2014-08-06 12:28:35 -0700 + Commit: cf8e7fd, github.com/apache/spark/pull/1801 + + [SPARK-2875] [PySpark] [SQL] handle null in schemaRDD() + Davies Liu + 2014-08-06 11:08:12 -0700 + Commit: 27a8d4c, github.com/apache/spark/pull/1802 + + [SPARK-2157] Enable tight firewall rules for Spark + Andrew Or , Andrew Ash + 2014-08-06 00:07:40 -0700 + Commit: 31090e4, github.com/apache/spark/pull/1777 + + [SPARK-1022][Streaming][HOTFIX] Fixed zookeeper dependency of Kafka + Tathagata Das + 2014-08-05 23:41:34 -0700 + Commit: 5b4bc84, github.com/apache/spark/pull/1797 + + [MLlib] Use this.type as return type in k-means' builder pattern + DB Tsai + 2014-08-05 23:32:29 -0700 + Commit: aec217a, github.com/apache/spark/pull/1796 + + SPARK-2294: fix locality inversion bug in TaskManager + CodingCat + 2014-08-05 23:02:58 -0700 + Commit: 1da2fdf, github.com/apache/spark/pull/1313 + + [SQL] Fix logging warn -> debug + Michael Armbrust + 2014-08-05 22:30:32 -0700 + Commit: 0482055, github.com/apache/spark/pull/1800 + + [SQL] Tighten the visibility of various SQLConf methods and renamed setter/getters + Reynold Xin + 2014-08-05 22:29:19 -0700 + Commit: 4f0b4f4, github.com/apache/spark/pull/1794 + + [SPARK-2806] core - upgrade to json4s-jackson 3.2.10 + Anand Avati + 2014-08-05 21:59:10 -0700 + Commit: 6da8f41, github.com/apache/spark/pull/1702 + + [SPARK-2866][SQL] Support attributes in ORDER BY that aren't in SELECT + Michael Armbrust + 2014-08-05 20:55:02 -0700 + Commit: 936f61e, github.com/apache/spark/pull/1795 + + [SPARK-2854][SQL] Finalize _acceptable_types in pyspark.sql + Yin Huai + 2014-08-05 18:56:10 -0700 + Commit: a10e1b0, github.com/apache/spark/pull/1793 + + [SPARK-2650][SQL] Try to partially fix SPARK-2650 by adjusting initial buffer size and reducing memory allocation + Cheng Lian + 2014-08-05 18:50:37 -0700 + Commit: 4233b02, github.com/apache/spark/pull/1769 + + [sql] rename project name in pom.xml of hive-thriftserver module + wangfei + 2014-08-05 18:30:02 -0700 + Commit: 152e36c, github.com/apache/spark/pull/1789 + + SPARK-2869 - Fix tiny bug in JdbcRdd for closing jdbc connection + Stephen Boesch , Stephen Boesch + 2014-08-05 18:18:08 -0700 + Commit: 58247a8, github.com/apache/spark/pull/1792 + + [SPARK-2550][MLLIB][APACHE SPARK] Support regularization and intercept in pyspark's linear methods + Michael Giannakopoulos + 2014-08-05 16:30:32 -0700 + Commit: 672904e, github.com/apache/spark/pull/1775 + + [SPARK-2503] Lower shuffle output buffer (spark.shuffle.file.buffer.kb) to 32KB. + Reynold Xin + 2014-08-05 16:24:50 -0700 + Commit: 0172277, github.com/apache/spark/pull/1781 + + [SPARK-2856] Decrease initial buffer size for Kryo to 64KB. + Reynold Xin + 2014-08-05 01:30:46 -0700 + Commit: 5081b0d, github.com/apache/spark/pull/1780 + + [SPARK-2864][MLLIB] fix random seed in word2vec; move model to local + Xiangrui Meng + 2014-08-05 16:22:41 -0700 + Commit: e77fa81, github.com/apache/spark/pull/1790 + + SPARK-1680: use configs for specifying environment variables on YARN + Thomas Graves + 2014-08-05 15:57:32 -0500 + Commit: 7b798e1, github.com/apache/spark/pull/1512 + + SPARK-2380: Support displaying accumulator values in the web UI + Patrick Wendell + 2014-08-05 13:08:23 -0700 + Commit: 46b6983, github.com/apache/spark/pull/1309 + + [SPARK-2859] Update url of Kryo project in related docs + Guancheng (G.C.) Chen + 2014-08-05 11:50:08 -0700 + Commit: 0f541ab, github.com/apache/spark/pull/1782 + + [SPARK-2860][SQL] Fix coercion of CASE WHEN. + Michael Armbrust + 2014-08-05 11:17:50 -0700 + Commit: 388ab53, github.com/apache/spark/pull/1785 + + SPARK-1890 and SPARK-1891- add admin and modify acls + Thomas Graves + 2014-08-05 12:52:52 -0500 + Commit: e3fe657, github.com/apache/spark/pull/1196 + + SPARK-1528 - spark on yarn, add support for accessing remote HDFS + Thomas Graves + 2014-08-05 12:48:26 -0500 + Commit: 6c0c65f, github.com/apache/spark/pull/1159 + + [SPARK-1022][Streaming] Add Kafka real unit test + jerryshao + 2014-08-05 10:40:28 -0700 + Commit: b92a450, github.com/apache/spark/pull/1751 + + [SPARK-1779] Throw an exception if memory fractions are not between 0 and 1 + wangfei , wangfei + 2014-08-05 00:51:07 -0700 + Commit: 075ba67, github.com/apache/spark/pull/714 + + [SPARK-2857] Correct properties to set Master / Worker ports + Andrew Or + 2014-08-05 00:39:07 -0700 + Commit: 12f99cf, github.com/apache/spark/pull/1779 + + SPARK-2711. Create a ShuffleMemoryManager to track memory for all spilling collections + Matei Zaharia + 2014-08-04 23:41:03 -0700 + Commit: d13d253, github.com/apache/spark/pull/1707 + + SPARK-2685. Update ExternalAppendOnlyMap to avoid buffer.remove() + Matei Zaharia + 2014-08-04 23:27:53 -0700 + Commit: a092285, github.com/apache/spark/pull/1773 + + [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext + Reynold Xin + 2014-08-04 20:39:18 -0700 + Commit: 4ed7b5a, github.com/apache/spark/pull/1772 + + [SPARK-1687] [PySpark] fix unit tests related to pickable namedtuple + Davies Liu + 2014-08-04 15:54:52 -0700 + Commit: 2225d18, github.com/apache/spark/pull/1771 + + SPARK-2792. Fix reading too much or too little data from each stream in ExternalMap / Sorter + Matei Zaharia + 2014-08-04 12:59:18 -0700 + Commit: aa7a48e, github.com/apache/spark/pull/1722 + + [SPARK-1687] [PySpark] pickable namedtuple + Davies Liu + 2014-08-04 12:13:41 -0700 + Commit: bfd2f39, github.com/apache/spark/pull/1623 + + [MLlib] [SPARK-2510]Word2Vec: Distributed Representation of Words + Liquan Pei , Xiangrui Meng , Liquan Pei + 2014-08-03 23:55:58 -0700 + Commit: 3823f6d, github.com/apache/spark/pull/1719 + + SPARK-2272 [MLlib] Feature scaling which standardizes the range of independent variables or features of data + DB Tsai + 2014-08-03 21:39:21 -0700 + Commit: 9aa1459, github.com/apache/spark/pull/1207 + + Fix some bugs with spaces in directory name. + Sarah Gerweck + 2014-08-03 19:47:05 -0700 + Commit: 2152e24, github.com/apache/spark/pull/1756 + + [SPARK-2810] upgrade to scala-maven-plugin 3.2.0 + Anand Avati + 2014-08-03 17:47:49 -0700 + Commit: 4784d24, github.com/apache/spark/pull/1711 + + [SPARK-1740] [PySpark] kill the python worker + Davies Liu + 2014-08-03 15:52:00 -0700 + Commit: a4cdb77, github.com/apache/spark/pull/1643 + + [SPARK-2783][SQL] Basic support for analyze in HiveContext + Yin Huai + 2014-08-03 14:54:41 -0700 + Commit: 7c6afda, github.com/apache/spark/pull/1741 + + [SPARK-2814][SQL] HiveThriftServer2 throws NPE when executing native commands + Cheng Lian + 2014-08-03 12:34:46 -0700 + Commit: 6ffdcc6, github.com/apache/spark/pull/1753 + + [SPARK-2784][SQL] Deprecate hql() method in favor of a config option, 'spark.sql.dialect' + Michael Armbrust + 2014-08-03 12:28:29 -0700 + Commit: c5ed1de, github.com/apache/spark/pull/1746 + + [SPARK-2197] [mllib] Java DecisionTree bug fix and easy-of-use + Joseph K. Bradley + 2014-08-03 10:36:52 -0700 + Commit: eaa9355, github.com/apache/spark/pull/1740 + + SPARK-2246: Add user-data option to EC2 scripts + Allan Douglas R. de Oliveira + 2014-08-03 10:25:59 -0700 + Commit: 162fc95, github.com/apache/spark/pull/1186 + + SPARK-2712 - Add a small note to maven doc that mvn package must happen ... + Stephen Boesch + 2014-08-03 10:19:04 -0700 + Commit: 1992175, github.com/apache/spark/pull/1615 + + [Minor] Fixes on top of #1679 + Andrew Or + 2014-08-02 22:00:46 -0700 + Commit: fb2a207, github.com/apache/spark/pull/1736 + + SPARK-2414 [BUILD] Add LICENSE entry for jquery + Sean Owen + 2014-08-02 21:55:56 -0700 + Commit: c137928, github.com/apache/spark/pull/1748 + + SPARK-2602 [BUILD] Tests steal focus under Java 6 + Sean Owen + 2014-08-02 21:44:19 -0700 + Commit: 0d47bb6, github.com/apache/spark/pull/1747 + + [SPARK-2739][SQL] Rename registerAsTable to registerTempTable + Michael Armbrust + 2014-08-02 18:27:04 -0700 + Commit: 5b30e00, github.com/apache/spark/pull/1743 + + [SPARK-2797] [SQL] SchemaRDDs don't support unpersist() + Yin Huai + 2014-08-02 17:55:22 -0700 + Commit: 5ef8282, github.com/apache/spark/pull/1745 + + [SPARK-2729][SQL] Added test case for SPARK-2729 + Cheng Lian + 2014-08-02 17:12:49 -0700 + Commit: 460fad8, github.com/apache/spark/pull/1738 + + [SPARK-2785][SQL] Remove assertions that throw when users try unsupported Hive commands. + Michael Armbrust + 2014-08-02 16:48:07 -0700 + Commit: 4230df4, github.com/apache/spark/pull/1742 + + [SPARK-2097][SQL] UDF Support + Michael Armbrust + 2014-08-02 16:33:48 -0700 + Commit: 3b9f25f, github.com/apache/spark/pull/1063 + + SPARK-2804: Remove scalalogging-slf4j dependency + GuoQiang Li + 2014-08-02 13:55:28 -0700 + Commit: 7924d72, github.com/apache/spark/pull/also + + [SPARK-1981] Add AWS Kinesis streaming support + Chris Fregly + 2014-08-02 13:35:35 -0700 + Commit: bb0ac6d, github.com/apache/spark/pull/1434 + + [SQL] Set outputPartitioning of BroadcastHashJoin correctly. + Yin Huai + 2014-08-02 13:16:41 -0700 + Commit: 91de0dc, github.com/apache/spark/pull/1735 + + [SPARK-2478] [mllib] DecisionTree Python API + Joseph K. Bradley + 2014-08-02 13:07:17 -0700 + Commit: 8d6ac2b, github.com/apache/spark/pull/1727 + + [HOTFIX] Do not throw NPE if spark.test.home is not set + Andrew Or + 2014-08-02 12:11:50 -0700 + Commit: e221108, github.com/apache/spark/pull/1739 + + MAINTENANCE: Automated closing of pull requests. + Patrick Wendell + 2014-08-02 01:26:16 -0700 + Commit: 87738bf, github.com/apache/spark/pull/706 + + HOTFIX: Fix concurrency issue in FlumePollingStreamSuite. + Patrick Wendell + 2014-08-02 01:11:03 -0700 + Commit: 44460ba + + HOTFIX: Fixing test error in maven for flume-sink. + Patrick Wendell + 2014-08-02 00:57:47 -0700 + Commit: 25cad6a + + [SPARK-1812] sql/catalyst - Provide explicit type information + Anand Avati + 2014-08-02 00:48:17 -0700 + Commit: 08c095b, github.com/apache/spark/pull/1709 + + [SPARK-2454] Do not ship spark home to Workers + Andrew Or + 2014-08-02 00:45:38 -0700 + Commit: 148af60, github.com/apache/spark/pull/1734 + + [SPARK-2316] Avoid O(blocks) operations in listeners + Andrew Or + 2014-08-01 23:56:24 -0700 + Commit: d934801, github.com/apache/spark/pull/1679 + + Revert "[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api" + Patrick Wendell + 2014-08-01 23:55:30 -0700 + Commit: dab3796 + + [SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api + GuoQiang Li + 2014-08-01 23:55:11 -0700 + Commit: adc8303, github.com/apache/spark/pull/1369 + + StatCounter on NumPy arrays [PYSPARK][SPARK-2012] + Jeremy Freeman + 2014-08-01 22:33:25 -0700 + Commit: 4bc3bb2, github.com/apache/spark/pull/1725 + + [SPARK-2801][MLlib]: DistributionGenerator renamed to RandomDataGenerator. RandomRDD is now of generic type + Burak + 2014-08-01 22:32:12 -0700 + Commit: fda4759, github.com/apache/spark/pull/1732 + + [SPARK-1580][MLLIB] Estimate ALS communication and computation costs. + Tor Myklebust , Xiangrui Meng + 2014-08-01 21:25:02 -0700 + Commit: e25ec06, github.com/apache/spark/pull/493 + + [SPARK-2550][MLLIB][APACHE SPARK] Support regularization and intercept in pyspark's linear methods. + Michael Giannakopoulos + 2014-08-01 21:00:31 -0700 + Commit: c281189, github.com/apache/spark/pull/1624 + + Streaming mllib [SPARK-2438][MLLIB] + Jeremy Freeman , freeman + 2014-08-01 20:10:26 -0700 + Commit: f6a1899, github.com/apache/spark/pull/1361 + + [SPARK-2764] Simplify daemon.py process structure + Josh Rosen + 2014-08-01 19:38:21 -0700 + Commit: e8e0fd6, github.com/apache/spark/pull/1680 + + [SPARK-2800]: Exclude scalastyle-output.xml Apache RAT checks + GuoQiang Li + 2014-08-01 19:35:16 -0700 + Commit: a38d3c9, github.com/apache/spark/pull/1729 + + [SPARK-2116] Load spark-defaults.conf from SPARK_CONF_DIR if set + Albert Chu + 2014-08-01 19:00:38 -0700 + Commit: 0da07da, github.com/apache/spark/pull/1059 + + [SPARK-2212][SQL] Hash Outer Join (follow-up bug fix). + Yin Huai + 2014-08-01 18:52:01 -0700 + Commit: 3822f33, github.com/apache/spark/pull/1721 + + [SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD + Davies Liu + 2014-08-01 18:47:41 -0700 + Commit: 880eabe, github.com/apache/spark/pull/1598 + + [SPARK-2796] [mllib] DecisionTree bug fix: ordered categorical features + Joseph K. Bradley + 2014-08-01 15:52:21 -0700 + Commit: 7058a53, github.com/apache/spark/pull/1720 + + [SPARK-2786][mllib] Python correlations + Doris Xin + 2014-08-01 15:02:17 -0700 + Commit: d88e695, github.com/apache/spark/pull/1713 + + SPARK-2791: Fix committing, reverting and state tracking in shuffle file consolidation + Aaron Davidson + 2014-08-01 13:57:19 -0700 + Commit: 78f2af5, github.com/apache/spark/pull/1678 + + [SPARK-2379] Fix the bug that streaming's receiver may fall into a dead loop + joyyoj + 2014-08-01 13:41:55 -0700 + Commit: b270309, github.com/apache/spark/pull/1694 + + SPARK-1612: Fix potential resource leaks + zsxwing + 2014-08-01 13:25:04 -0700 + Commit: f5d9bea, github.com/apache/spark/pull/535 + + [SPARK-2490] Change recursive visiting on RDD dependencies to iterative approach + Liang-Chi Hsieh + 2014-08-01 12:12:30 -0700 + Commit: baf9ce1, github.com/apache/spark/pull/1418 + + [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions. + Aaron Staple + 2014-08-01 12:04:04 -0700 + Commit: eb5bdca, github.com/apache/spark/pull/1362 + + [SQL] Documentation: Explain cacheTable command + CrazyJvm + 2014-08-01 11:46:13 -0700 + Commit: c82fe47, github.com/apache/spark/pull/1681 + + [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed. + Cheng Hao + 2014-08-01 11:42:05 -0700 + Commit: c0b47ba, github.com/apache/spark/pull/1686 + + [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilder + chutium + 2014-08-01 11:31:44 -0700 + Commit: 580c701, github.com/apache/spark/pull/1636 + + [SQL][SPARK-2212]Hash Outer Join + Cheng Hao + 2014-08-01 11:27:12 -0700 + Commit: 4415722, github.com/apache/spark/pull/1147 + + [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up). + Yin Huai + 2014-08-01 11:14:53 -0700 + Commit: c41fdf0, github.com/apache/spark/pull/1712 + + SPARK-2099. Report progress while task is running. + Sandy Ryza + 2014-08-01 11:08:39 -0700 + Commit: 8d338f6, github.com/apache/spark/pull/1056 + + [HOTFIX] downgrade breeze version to 0.7 + Xiangrui Meng + 2014-08-01 10:00:46 -0700 + Commit: 5328c0a, github.com/apache/spark/pull/1718 + + [SPARK-1997] update breeze to version 0.8.1 + witgo + 2014-08-01 07:47:44 -0700 + Commit: 0dacb1a, github.com/apache/spark/pull/940 + + SPARK-2768 [MLLIB] Add product, user recommend method to MatrixFactorizationModel + Sean Owen + 2014-08-01 07:32:53 -0700 + Commit: 82d209d, github.com/apache/spark/pull/1687 + + [SPARK-2103][Streaming] Change to ClassTag for KafkaInputDStream and fix reflection issue + jerryshao + 2014-08-01 04:32:46 -0700 + Commit: a32f0fb, github.com/apache/spark/pull/1508 + + [Spark 2557] fix LOCAL_N_REGEX in createTaskScheduler and make local-n and local-n-failures consistent + Ye Xianjin + 2014-08-01 00:34:39 -0700 + Commit: 284771e, github.com/apache/spark/pull/1464 + + SPARK-2134: Report metrics before application finishes + Rahul Singhal + 2014-08-01 00:33:15 -0700 + Commit: f1957e1, github.com/apache/spark/pull/1076 + + SPARK-983. Support external sorting in sortByKey() + Matei Zaharia + 2014-08-01 00:16:18 -0700 + Commit: 72e3369, github.com/apache/spark/pull/931 + + [SPARK-2670] FetchFailedException should be thrown when local fetch has failed + Kousuke Saruta + 2014-08-01 00:01:30 -0700 + Commit: 8ff4417, github.com/apache/spark/pull/1578 + + SPARK-2738. Remove redundant imports in BlockManagerSuite + Sandy Ryza + 2014-07-31 23:12:38 -0700 + Commit: cb9e7d5, github.com/apache/spark/pull/1642 + + SPARK-2632, SPARK-2576. Fixed by only importing what is necessary during class definition. + Prashant Sharma , Yin Huai , Prashant Sharma + 2014-07-31 22:57:13 -0700 + Commit: 1499101, github.com/apache/spark/pull/1635 + + [SPARK-2702][Core] Upgrade Tachyon dependency to 0.5.0 + Haoyuan Li + 2014-07-31 22:53:42 -0700 + Commit: 2cdc3e5, github.com/apache/spark/pull/1651 + + [SPARK-2782][mllib] Bug fix for getRanks in SpearmanCorrelation + Doris Xin + 2014-07-31 21:23:35 -0700 + Commit: c475540, github.com/apache/spark/pull/1710 + + [SPARK-2777][MLLIB] change ALS factors storage level to MEMORY_AND_DISK + Xiangrui Meng + 2014-07-31 21:14:08 -0700 + Commit: b190083, github.com/apache/spark/pull/1700 + + SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6 + GuoQiang Li + 2014-07-31 21:06:57 -0700 + Commit: 9998efa, github.com/apache/spark/pull/1683 + + [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map instead of scala.collection.immutable.Map + Yin Huai + 2014-07-31 21:02:11 -0700 + Commit: 9632719, github.com/apache/spark/pull/1705 + + [SPARK-2756] [mllib] Decision tree bug fixes + Joseph K. Bradley + 2014-07-31 20:51:48 -0700 + Commit: b124de5, github.com/apache/spark/pull/1673 + + [SPARK-2724] Python version of RandomRDDGenerators + Doris Xin + 2014-07-31 20:32:57 -0700 + Commit: d843014, github.com/apache/spark/pull/1628 + + [SPARK-2531 & SPARK-2436] [SQL] Optimize the BuildSide when planning BroadcastNestedLoopJoin. + Zongheng Yang + 2014-07-31 19:32:16 -0700 + Commit: 8f51491, github.com/apache/spark/pull/1448 + + SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark + Aaron Davidson + 2014-07-31 15:31:53 -0700 + Commit: ef4ff00, github.com/apache/spark/pull/1503 + + SPARK-2740: allow user to specify ascending and numPartitions for sortBy... + Rui Li + 2014-07-31 15:07:26 -0700 + Commit: 492a195, github.com/apache/spark/pull/1645 + + Docs: monitoring, streaming programming guide + kballou + 2014-07-31 14:58:52 -0700 + Commit: cc82050, github.com/apache/spark/pull/1662 + + Improvements to merge_spark_pr.py + Josh Rosen + 2014-07-31 14:35:09 -0700 + Commit: e021362, github.com/apache/spark/pull/1668 + + [SPARK-2523] [SQL] Hadoop table scan bug fixing (fix failing Jenkins maven test) + Yin Huai + 2014-07-31 13:05:24 -0700 + Commit: 49b3612, github.com/apache/spark/pull/1669 + + [SPARK-2511][MLLIB] add HashingTF and IDF + Xiangrui Meng + 2014-07-31 12:55:00 -0700 + Commit: dc0865b, github.com/apache/spark/pull/1671 + + SPARK-2646. log4j initialization not quite compatible with log4j 2.x + Sean Owen + 2014-07-31 12:26:36 -0700 + Commit: e5749a1, github.com/apache/spark/pull/1547 + + SPARK-2749 [BUILD] Part 2. Fix a follow-on scalastyle error + Sean Owen + 2014-07-31 12:18:40 -0700 + Commit: 4dbabb3, github.com/apache/spark/pull/1690 + + SPARK-2664. Deal with `--conf` options in spark-submit that relate to fl... + Sandy Ryza + 2014-07-31 11:51:20 -0700 + Commit: f68105d, github.com/apache/spark/pull/1665 + + SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD + Aaron Davidson + 2014-07-31 11:35:38 -0700 + Commit: f193312, github.com/apache/spark/pull/973 + + [SPARK-2397][SQL] Deprecate LocalHiveContext + Michael Armbrust + 2014-07-31 11:26:43 -0700 + Commit: 72cfb13, github.com/apache/spark/pull/1641 + + [SPARK-2743][SQL] Resolve original attributes in ParquetTableScan + Michael Armbrust + 2014-07-31 11:15:25 -0700 + Commit: 3072b96, github.com/apache/spark/pull/1647 + + [SPARK-2762] SparkILoop leaks memory in multi-repl configurations + Timothy Hunter + 2014-07-31 10:25:40 -0700 + Commit: 92ca910, github.com/apache/spark/pull/1674 + + automatically set master according to `spark.master` in `spark-defaults.... + CrazyJvm + 2014-07-30 23:37:25 -0700 + Commit: 669e3f0, github.com/apache/spark/pull/1644 + + [SPARK-2497] Included checks for module symbols too. + Prashant Sharma + 2014-07-30 22:46:30 -0700 + Commit: 5a110da, github.com/apache/spark/pull/1463 + + [SPARK-2737] Add retag() method for changing RDDs' ClassTags. + Josh Rosen + 2014-07-30 22:40:57 -0700 + Commit: 4fb2593, github.com/apache/spark/pull/1639 + + [SPARK-2340] Resolve event logging and History Server paths properly + Andrew Or + 2014-07-30 21:57:32 -0700 + Commit: a7c305b, github.com/apache/spark/pull/1280 + + Required AM memory is "amMem", not "args.amMemory" + derek ma + 2014-07-30 21:37:59 -0700 + Commit: 118c1c4, github.com/apache/spark/pull/1494 + + [SPARK-2758] UnionRDD's UnionPartition should not reference parent RDDs + Reynold Xin + 2014-07-30 21:30:13 -0700 + Commit: 894d48f, github.com/apache/spark/pull/1675 + + SPARK-2045 Sort-based shuffle + Matei Zaharia + 2014-07-30 18:07:59 -0700 + Commit: e966284, github.com/apache/spark/pull/1499 + + Update DecisionTreeRunner.scala + strat0sphere + 2014-07-30 17:57:50 -0700 + Commit: da50176, github.com/apache/spark/pull/1676 + + SPARK-2341 [MLLIB] loadLibSVMFile doesn't handle regression datasets + Sean Owen + 2014-07-30 17:34:32 -0700 + Commit: e9b275b, github.com/apache/spark/pull/1663 + + [SPARK-2734][SQL] Remove tables from cache when DROP TABLE is run. + Michael Armbrust + 2014-07-30 17:30:51 -0700 + Commit: 88a519d, github.com/apache/spark/pull/1650 + + SPARK-2741 - Publish version of spark assembly which does not contain Hive + Brock Noland + 2014-07-30 17:04:30 -0700 + Commit: 2ac37db, github.com/apache/spark/pull/1667 + + SPARK-2749 [BUILD]. Spark SQL Java tests aren't compiling in Jenkins' Maven builds; missing junit:junit dep + Sean Owen + 2014-07-30 15:04:33 -0700 + Commit: 6ab96a6, github.com/apache/spark/pull/1660 + + Properly pass SBT_MAVEN_PROFILES into sbt. + Reynold Xin + 2014-07-30 14:31:20 -0700 + Commit: 2f4b170 + + Set AMPLAB_JENKINS_BUILD_PROFILE. + Reynold Xin + 2014-07-30 14:08:24 -0700 + Commit: 1097327 + + Wrap JAR_DL in dev/check-license. + Reynold Xin + 2014-07-30 13:42:43 -0700 + Commit: 7c7ce54 + + [SPARK-2024] Add saveAsSequenceFile to PySpark + Kan Zhang + 2014-07-30 13:19:05 -0700 + Commit: 94d1f46, github.com/apache/spark/pull/1338 + + dev/check-license wrap folders in quotes. + Reynold Xin + 2014-07-30 13:17:14 -0700 + Commit: 437dc8c + + [SQL] Fix compiling of catalyst docs. + Michael Armbrust + 2014-07-30 13:11:09 -0700 + Commit: 2248891, github.com/apache/spark/pull/1653 + + More wrapping FWDIR in quotes. + Reynold Xin + 2014-07-30 13:04:20 -0700 + Commit: 0feb349 + + Wrap FWDIR in quotes in dev/check-license. + Reynold Xin + 2014-07-30 12:33:42 -0700 + Commit: 95cf203 + + Wrap FWDIR in quotes. + Reynold Xin + 2014-07-30 12:24:35 -0700 + Commit: f2eb84f + + [SPARK-2746] Set SBT_MAVEN_PROFILES only when it is not set explicitly by the user. + Reynold Xin + 2014-07-30 11:45:24 -0700 + Commit: ff511ba, github.com/apache/spark/pull/1655 + + [SPARK-2544][MLLIB] Improve ALS algorithm resource usage + GuoQiang Li , witgo + 2014-07-30 11:00:11 -0700 + Commit: fc47bb6, github.com/apache/spark/pull/929 + + Avoid numerical instability + Naftali Harris + 2014-07-30 09:56:59 -0700 + Commit: e3d85b7, github.com/apache/spark/pull/1652 + + [SPARK-2747] git diff --dirstat can miss sql changes and not run Hive tests + Reynold Xin + 2014-07-30 09:28:53 -0700 + Commit: 3bc3f18, github.com/apache/spark/pull/1656 + + [SPARK-2521] Broadcast RDD object (instead of sending it along with every task) + Reynold Xin + 2014-07-30 09:27:43 -0700 + Commit: 774142f, github.com/apache/spark/pull/1498 + + SPARK-2748 [MLLIB] [GRAPHX] Loss of precision for small arguments to Math.exp, Math.log + Sean Owen + 2014-07-30 08:55:15 -0700 + Commit: ee07541, github.com/apache/spark/pull/1659 + + SPARK-2543: Allow user to set maximum Kryo buffer size + Koert Kuipers + 2014-07-30 00:18:59 -0700 + Commit: 7c5fc28, github.com/apache/spark/pull/735 + + [SPARK-2179][SQL] Public API for DataTypes and Schema + Yin Huai + 2014-07-30 00:15:31 -0700 + Commit: 7003c16, github.com/apache/spark/pull/1346 + + [SPARK-2260] Fix standalone-cluster mode, which was broken + Andrew Or + 2014-07-29 23:52:09 -0700 + Commit: 4ce92cc, github.com/apache/spark/pull/1538 + + [SQL] Handle null values in debug() + Michael Armbrust + 2014-07-29 22:42:54 -0700 + Commit: 077f633, github.com/apache/spark/pull/1646 + + [SPARK-2568] RangePartitioner should run only one job if data is balanced + Xiangrui Meng , Reynold Xin + 2014-07-29 22:16:20 -0700 + Commit: 2e6efca, github.com/apache/spark/pull/1562 + + [SPARK-2054][SQL] Code Generation for Expression Evaluation + Michael Armbrust + 2014-07-29 20:58:05 -0700 + Commit: 8446746, github.com/apache/spark/pull/993 + + [SPARK-2305] [PySpark] Update Py4J to version 0.8.2.1 + Josh Rosen + 2014-07-29 19:02:06 -0700 + Commit: 22649b6, github.com/apache/spark/pull/1626 + + [SPARK-2631][SQL] Use SQLConf to configure in-memory columnar caching + Michael Armbrust + 2014-07-29 18:20:51 -0700 + Commit: 86534d0, github.com/apache/spark/pull/1638 + + [SPARK-2716][SQL] Don't check resolved for having filters. + Michael Armbrust + 2014-07-29 18:14:20 -0700 + Commit: 39b8193, github.com/apache/spark/pull/1640 + + MAINTENANCE: Automated closing of pull requests. + Patrick Wendell + 2014-07-29 17:52:48 -0700 + Commit: 2c35666, github.com/apache/spark/pull/740 + + [SPARK-2393][SQL] Cost estimation optimization framework for Catalyst logical plans & sample usage. + Zongheng Yang + 2014-07-29 15:32:50 -0700 + Commit: c7db274, github.com/apache/spark/pull/1238 + + [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size + Doris Xin , Xiangrui Meng + 2014-07-29 12:49:44 -0700 + Commit: dc96536, github.com/apache/spark/pull/1025 + + [SPARK-2674] [SQL] [PySpark] support datetime type for SchemaRDD + Davies Liu + 2014-07-29 12:31:39 -0700 + Commit: f0d880e, github.com/apache/spark/pull/1601 + + [SPARK-2730][SQL] When retrieving a value from a Map, GetItem evaluates key twice + Yin Huai + 2014-07-29 12:23:34 -0700 + Commit: e364348, github.com/apache/spark/pull/1637 + + [SQL]change some test lists + Daoyuan + 2014-07-29 12:22:48 -0700 + Commit: 0c5c6a6, github.com/apache/spark/pull/1634 + + [STREAMING] SPARK-1729. Make Flume pull data from source, rather than the current pu... + Hari Shreedharan , Hari Shreedharan , Tathagata Das , harishreedharan + 2014-07-29 11:11:29 -0700 + Commit: 800ecff, github.com/apache/spark/pull/807 + + Minor indentation and comment typo fixes. + Aaron Staple + 2014-07-29 01:35:26 -0700 + Commit: fc4d057, github.com/apache/spark/pull/1630 + + [SPARK-2174][MLLIB] treeReduce and treeAggregate + Xiangrui Meng + 2014-07-29 01:16:41 -0700 + Commit: 20424da, github.com/apache/spark/pull/1110 + + [SPARK-2726] and [SPARK-2727] Remove SortOrder and do in-place sort. + Reynold Xin + 2014-07-29 01:12:44 -0700 + Commit: 96ba04b, github.com/apache/spark/pull/1631 + + [SPARK-791] [PySpark] fix pickle itemgetter with cloudpickle + Davies Liu + 2014-07-29 01:02:18 -0700 + Commit: 92ef026, github.com/apache/spark/pull/1627 + + [SPARK-2580] [PySpark] keep silent in worker if JVM close the socket + Davies Liu + 2014-07-29 00:15:45 -0700 + Commit: ccd5ab5, github.com/apache/spark/pull/1625 + + Excess judgment + Yadong Qi + 2014-07-28 21:39:02 -0700 + Commit: 16ef4d1, github.com/apache/spark/pull/1629 + + Use commons-lang3 in SignalLogger rather than commons-lang + Aaron Davidson + 2014-07-28 13:37:44 -0700 + Commit: 39ab87b, github.com/apache/spark/pull/1621 + + [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (with Maven profile fix) + Cheng Lian + 2014-07-28 12:07:30 -0700 + Commit: a7a9d14, github.com/apache/spark/pull/1620 + + [SPARK-2479][MLlib] Comparing floating-point numbers using relative error in UnitTests + DB Tsai + 2014-07-28 11:34:19 -0700 + Commit: 255b56f, github.com/apache/spark/pull/1425 + + [SPARK-2523] [SQL] Hadoop table scan bug fixing + Cheng Hao + 2014-07-28 10:59:53 -0700 + Commit: 2b8d89e, github.com/apache/spark/pull/1439 + + [SPARK-1550] [PySpark] Allow SparkContext creation after failed attempts + Josh Rosen + 2014-07-27 22:54:43 -0700 + Commit: a7d145e, github.com/apache/spark/pull/1606 + + SPARK-2651: Add maven scalastyle plugin + Rahul Singhal + 2014-07-27 18:50:32 -0700 + Commit: d7eac4c, github.com/apache/spark/pull/1550 + + Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server" + Patrick Wendell + 2014-07-27 18:46:58 -0700 + Commit: e5bbce9 + + [SPARK-2514] [mllib] Random RDD generator + Doris Xin + 2014-07-27 16:16:39 -0700 + Commit: 81fcdd2, github.com/apache/spark/pull/1520 + + [SPARK-1777] Prevent OOMs from single partitions + Andrew Or + 2014-07-27 16:08:16 -0700 + Commit: ecf30ee, github.com/apache/spark/pull/1165 + + [SPARK-2410][SQL] Merging Hive Thrift/JDBC server + Cheng Lian + 2014-07-27 13:03:38 -0700 + Commit: f6ff2a6, github.com/apache/spark/pull/1600 + + [SPARK-2705][CORE] Fixed stage description in stage info page + Cheng Lian + 2014-07-27 12:35:21 -0700 + Commit: 2bbf235, github.com/apache/spark/pull/1524 + + SPARK-2684: Update ExternalAppendOnlyMap to take an iterator as input + Matei Zaharia + 2014-07-27 11:20:20 -0700 + Commit: 9857053, github.com/apache/spark/pull/1607 + + [SPARK-2679] [MLLib] Ser/De for Double + Doris Xin + 2014-07-27 07:21:07 -0700 + Commit: 3a69c72, github.com/apache/spark/pull/1581 + + [SPARK-2361][MLLIB] Use broadcast instead of serializing data directly into task closure + Xiangrui Meng + 2014-07-26 22:56:07 -0700 + Commit: aaf2b73, github.com/apache/spark/pull/1427 + + SPARK-2680: Lower spark.shuffle.memoryFraction to 0.2 by default + Matei Zaharia + 2014-07-26 22:44:17 -0700 + Commit: b547f69, github.com/apache/spark/pull/1593 + + [SPARK-2601] [PySpark] Fix Py4J error when transforming pickleFiles + Josh Rosen + 2014-07-26 17:37:05 -0700 + Commit: ba46bbe, github.com/apache/spark/pull/1605 + + [SPARK-2704] Name threads in ConnectionManager and mark them as daemon. + Reynold Xin + 2014-07-26 15:00:32 -0700 + Commit: 1290164, github.com/apache/spark/pull/1604 + + [SPARK-2279] Added emptyRDD method to Java API + bpaulin + 2014-07-26 10:27:09 -0700 + Commit: c183b92, github.com/apache/spark/pull/1597 + + [SPARK-2652] [PySpark] Turning some default configs for PySpark + Davies Liu + 2014-07-26 01:07:08 -0700 + Commit: 75663b5, github.com/apache/spark/pull/1568 + + [SPARK-2696] Reduce default value of spark.serializer.objectStreamReset + Hossein + 2014-07-26 01:04:56 -0700 + Commit: 66f26a4, github.com/apache/spark/pull/1595 + + [SPARK-1458] [PySpark] Expose sc.version in Java and PySpark + Josh Rosen + 2014-07-26 00:54:05 -0700 + Commit: cf3e9fd, github.com/apache/spark/pull/1596 + + [SPARK-2659][SQL] Fix division semantics for hive + Michael Armbrust + 2014-07-25 19:17:49 -0700 + Commit: 8904791, github.com/apache/spark/pull/1557 + + Part of [SPARK-2456] Removed some HashMaps from DAGScheduler by storing information in Stage. + Reynold Xin + 2014-07-25 18:45:02 -0700 + Commit: 9d8666c, github.com/apache/spark/pull/1561 + + Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server" + Michael Armbrust + 2014-07-25 15:36:57 -0700 + Commit: afd757a, github.com/apache/spark/pull/1594 + + [SPARK-1726] [SPARK-2567] Eliminate zombie stages in UI. + Kay Ousterhout + 2014-07-25 15:14:13 -0700 + Commit: 37ad3b7, github.com/apache/spark/pull/1566 + + [SPARK-2125] Add sort flag and move sort into shuffle implementations + jerryshao + 2014-07-25 14:34:38 -0700 + Commit: 47b6b38, github.com/apache/spark/pull/1210 + + [SQL]Update HiveMetastoreCatalog.scala + baishuo(白硕) + 2014-07-25 13:59:45 -0700 + Commit: ab3c6a4, github.com/apache/spark/pull/1569 + + [SPARK-2682] Javadoc generated from Scala source code is not in javadoc's index + Yin Huai + 2014-07-25 13:00:13 -0700 + Commit: a19d8c8, github.com/apache/spark/pull/1584 + + [SPARK-2410][SQL] Merging Hive Thrift/JDBC server + Cheng Lian + 2014-07-25 12:20:49 -0700 + Commit: 06dc0d2, github.com/apache/spark/pull/1399 + + [SPARK-2683] unidoc failed because org.apache.spark.util.CallSite uses Java keywords as value names + Yin Huai + 2014-07-25 11:14:51 -0700 + Commit: 32bcf9a, github.com/apache/spark/pull/1585 + + replace println to log4j + fireflyc + 2014-07-25 10:47:52 -0700 + Commit: a2715cc, github.com/apache/spark/pull/1372 + + [SPARK-2665] [SQL] Add EqualNS & Unit Tests + Cheng Hao + 2014-07-25 01:30:22 -0700 + Commit: 184aa1c, github.com/apache/spark/pull/1570 + + [SPARK-2529] Clean closures in foreach and foreachPartition. + Reynold Xin + 2014-07-25 01:10:05 -0700 + Commit: eb82abd, github.com/apache/spark/pull/1583 + + SPARK-2657 Use more compact data structures than ArrayBuffer in groupBy & cogroup + Matei Zaharia + 2014-07-25 00:32:32 -0700 + Commit: 8529ced, github.com/apache/spark/pull/1555 + + [SPARK-2656] Python version of stratified sampling + Doris Xin + 2014-07-24 23:42:08 -0700 + Commit: 2f75a4a, github.com/apache/spark/pull/1554 + + [SPARK-2538] [PySpark] Hash based disk spilling aggregation + Davies Liu + 2014-07-24 22:53:47 -0700 + Commit: 14174ab, github.com/apache/spark/pull/1460 + + [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default + Prashant Sharma + 2014-07-24 18:15:37 -0700 + Commit: eff9714, github.com/apache/spark/pull/1051 + + [SPARK-2464][Streaming] Fixed Twitter stream stopping bug + Tathagata Das + 2014-07-24 15:59:09 -0700 + Commit: a45d548, github.com/apache/spark/pull/1577 + + SPARK-2250: show stage RDDs in UI + Neville Li + 2014-07-24 14:13:00 -0700 + Commit: fec641b, github.com/apache/spark/pull/1188 + + [SPARK-2037]: yarn client mode doesn't support spark.yarn.max.executor.failures + GuoQiang Li + 2014-07-24 14:46:10 -0500 + Commit: 323a83c, github.com/apache/spark/pull/1180 + + [SPARK-2479 (partial)][MLLIB] fix binary metrics unit tests + Xiangrui Meng + 2014-07-24 12:37:02 -0700 + Commit: c960b50, github.com/apache/spark/pull/1576 + + [SPARK-2603][SQL] Remove unnecessary toMap and toList in converting Java collections to Scala collections JsonRDD.scala + Yin Huai + 2014-07-24 11:19:19 -0700 + Commit: b352ef1, github.com/apache/spark/pull/1504 + + [Build] SPARK-2619: Configurable filemode for the spark/bin folder in debian package + tzolov + 2014-07-24 11:12:25 -0700 + Commit: 9fd1414, github.com/apache/spark/pull/1531 + + SPARK-2150: Provide direct link to finished application UI in yarn resou... + Rahul Singhal + 2014-07-24 09:31:04 -0500 + Commit: 46e224a, github.com/apache/spark/pull/1094 + + [SPARK-2661][bagel]unpersist old processed rdd + Daoyuan + 2014-07-24 00:09:36 -0700 + Commit: 42dfab7, github.com/apache/spark/pull/1519 + + SPARK-2310. Support arbitrary Spark properties on the command line with ... + Sandy Ryza + 2014-07-23 23:09:25 -0700 + Commit: e34922a, github.com/apache/spark/pull/1253 + + [SPARK-2658][SQL] Add rule for true = 1. + Michael Armbrust + 2014-07-23 22:52:49 -0700 + Commit: 78d18fd, github.com/apache/spark/pull/1556 + + SPARK-2662: Fix NPE for JsonProtocol + GuoQiang Li + 2014-07-23 22:50:39 -0700 + Commit: 9e7725c, github.com/apache/spark/pull/1511 + + Replace RoutingTableMessage with pair + Ankur Dave + 2014-07-23 20:11:28 -0700 + Commit: 2d25e34, github.com/apache/spark/pull/1553 + + [SPARK-2484][SQL] Build should not run hivecompatibility tests by default. + witgo + 2014-07-23 18:17:05 -0700 + Commit: 60f0ae3, github.com/apache/spark/pull/1403 + + [SPARK-2549] Functions defined inside of other functions trigger failures + Prashant Sharma + 2014-07-23 17:12:28 -0700 + Commit: 9b76332, github.com/apache/spark/pull/1510 + + [SPARK-2102][SQL][CORE] Add option for kryo registration required and use a resource pool in Spark SQL for Kryo instances. + Ian O Connell + 2014-07-23 16:30:06 -0700 + Commit: efdaeb1, github.com/apache/spark/pull/1377 + + [SPARK-2569][SQL] Fix shipping of TEMPORARY hive UDFs. + Michael Armbrust + 2014-07-23 16:26:55 -0700 + Commit: 1871574, github.com/apache/spark/pull/1552 + + SPARK-2226: [SQL] transform HAVING clauses with aggregate expressions that aren't in the aggregation list + William Benton + 2014-07-23 16:25:32 -0700 + Commit: e060d3e, github.com/apache/spark/pull/1497 + + SPARK-2277: clear host->rack info properly + Rui Li + 2014-07-23 16:23:24 -0700 + Commit: 91903e0, github.com/apache/spark/pull/1454 + + [SPARK-2588][SQL] Add some more DSLs. + Takuya UESHIN + 2014-07-23 14:47:23 -0700 + Commit: 1b790cf, github.com/apache/spark/pull/1491 + + [CORE] SPARK-2640: In "local[N]", free cores of the only executor should be touched by "spark.task.cpus" for every finish/start-up of tasks. + woshilaiceshide + 2014-07-23 11:05:41 -0700 + Commit: f776bc9, github.com/apache/spark/pull/1544 + + [SPARK-2609] Log thread ID when spilling ExternalAppendOnlyMap + Andrew Or + 2014-07-23 10:31:45 -0700 + Commit: 2592111, github.com/apache/spark/pull/1517 + + [SPARK-2617] Correct doc and usages of preservesPartitioning + Xiangrui Meng + 2014-07-23 00:58:55 -0700 + Commit: 4c7243e, github.com/apache/spark/pull/1526 + + Remove GraphX MessageToPartition for compatibility with sort-based shuffle + Ankur Dave + 2014-07-22 22:18:30 -0700 + Commit: 6c2be93, github.com/apache/spark/pull/1537 + + [YARN] SPARK-2577: File upload to viewfs is broken due to mount point re... + Gera Shegalov + 2014-07-22 21:05:12 -0500 + Commit: 02e4572, github.com/apache/spark/pull/1483 + + [YARN][SPARK-2606]:In some cases,the spark UI pages display incorrect + GuoQiang Li + 2014-07-22 20:34:40 -0500 + Commit: ddadf1b, github.com/apache/spark/pull/1501 + + Graphx example + CrazyJvm + 2014-07-22 18:14:44 -0700 + Commit: 5f7b991, github.com/apache/spark/pull/1523 + + [SPARK-2615] [SQL] Add Equal Sign "==" Support for HiveQl + Cheng Hao + 2014-07-22 18:13:28 -0700 + Commit: 79fe763, github.com/apache/spark/pull/1522 + + SPARK-2047: Introduce an in-mem Sorter, and use it to reduce mem usage + Aaron Davidson + 2014-07-22 11:58:53 -0700 + Commit: 85d3596, github.com/apache/spark/pull/1502 + + [MLLIB] make Mima ignore updateFeatures (private) in ALS + Xiangrui Meng + 2014-07-22 11:45:37 -0700 + Commit: 1407871, github.com/apache/spark/pull/1533 + + [SPARK-2612] [mllib] Fix data skew in ALS + peng.zhang + 2014-07-22 02:39:07 -0700 + Commit: 75db174, github.com/apache/spark/pull/1521 + + [SPARK-2452] Create a new valid for each instead of using lineId. + Prashant Sharma + 2014-07-22 00:38:26 -0700 + Commit: 81fec99, github.com/apache/spark/pull/1441 + + [SPARK-2470] PEP8 fixes to PySpark + Nicholas Chammas , nchammas + 2014-07-21 22:30:53 -0700 + Commit: 5d16d5b, github.com/apache/spark/pull/1505 + + [SPARK-2086] Improve output of toDebugString to make shuffle boundaries more clear + Gregory Owen + 2014-07-21 18:55:01 -0700 + Commit: c3462c6, github.com/apache/spark/pull/1364 + + [SPARK-2561][SQL] Fix apply schema + Michael Armbrust + 2014-07-21 18:18:17 -0700 + Commit: 511a731, github.com/apache/spark/pull/1470 + + [SPARK-2434][MLlib]: Warning messages that point users to original MLlib implementations added to Examples + Burak + 2014-07-21 17:03:40 -0700 + Commit: a4d6020, github.com/apache/spark/pull/1515 + + Fix flakey HiveQuerySuite test + Aaron Davidson + 2014-07-21 14:35:15 -0700 + Commit: abeacff, github.com/apache/spark/pull/1514 + + [SPARK-2494] [PySpark] make hash of None consistant cross machines + Davies Liu + 2014-07-21 11:59:54 -0700 + Commit: 872538c, github.com/apache/spark/pull/1371 + + SPARK-1707. Remove unnecessary 3 second sleep in YarnClusterScheduler + Sandy Ryza + 2014-07-21 13:15:46 -0500 + Commit: f89cf65, github.com/apache/spark/pull/634 + + [SPARK-2190][SQL] Specialized ColumnType for Timestamp + Cheng Lian + 2014-07-21 00:46:28 -0700 + Commit: cd273a2, github.com/apache/spark/pull/1440 + + [SPARK-1945][MLLIB] Documentation Improvements for Spark 1.0 + Michael Giannakopoulos + 2014-07-20 20:48:44 -0700 + Commit: db56f2d, github.com/apache/spark/pull/1311 + + Improve scheduler delay tooltip. + Kay Ousterhout + 2014-07-20 20:18:18 -0700 + Commit: f6e7302, github.com/apache/spark/pull/1488 + + [SPARK-2552][MLLIB] stabilize logistic function in pyspark + Xiangrui Meng + 2014-07-20 18:40:36 -0700 + Commit: b86db51, github.com/apache/spark/pull/1493 + + SPARK-2564. ShuffleReadMetrics.totalBlocksRead is redundant + Sandy Ryza + 2014-07-20 14:45:34 -0700 + Commit: 9564f85, github.com/apache/spark/pull/1474 + + [SPARK-2495][MLLIB] remove private[mllib] from linear models' constructors + Xiangrui Meng + 2014-07-20 13:04:59 -0700 + Commit: 1b10b81, github.com/apache/spark/pull/1492 + + [SPARK-2598] RangePartitioner's binary search does not use the given Ordering + Reynold Xin + 2014-07-20 11:06:06 -0700 + Commit: fa51b0f, github.com/apache/spark/pull/1500 + + SPARK-2519 part 2. Remove pattern matching on Tuple2 in critical section... + Sandy Ryza + 2014-07-20 01:24:32 -0700 + Commit: 98ab411, github.com/apache/spark/pull/1447 + + [SPARK-2524] missing document about spark.deploy.retainedDrivers + lianhuiwang , Wang Lianhui , unknown + 2014-07-19 20:46:59 -0700 + Commit: 4da01e3, github.com/apache/spark/pull/1443 + + SPARK-2587: Fix error message in make-distribution.sh + Mark Wagner + 2014-07-19 20:24:13 -0700 + Commit: c119498, github.com/apache/spark/pull/1489 + + Typo fix to the programming guide in the docs + Cesar Arevalo + 2014-07-19 20:20:07 -0700 + Commit: 0d01e85, github.com/apache/spark/pull/1495 + + SPARK-2596 HOTFIX: Deal with non-existent JIRAs. + Patrick Wendell + 2014-07-19 18:24:21 -0700 + Commit: d39e3b9 + + SPARK-2596 A tool for mirroring github pull requests on JIRA. + Patrick Wendell + 2014-07-19 18:19:08 -0700 + Commit: 49e4727, github.com/apache/spark/pull/1496 + + Revert "[SPARK-2521] Broadcast RDD object (instead of sending it along with every task)." + Reynold Xin + 2014-07-19 16:56:22 -0700 + Commit: 1efb369 + + SPARK-2407: Added Parser of SQL SUBSTR() + chutium + 2014-07-19 11:04:41 -0500 + Commit: 2a73211, github.com/apache/spark/pull/1442 + + put 'curRequestSize = 0' after 'logDebug' it + Lijie Xu + 2014-07-19 01:27:26 -0700 + Commit: 805f329, github.com/apache/spark/pull/1477 + + [SPARK-2521] Broadcast RDD object (instead of sending it along with every task). + Reynold Xin + 2014-07-18 23:52:47 -0700 + Commit: 7b8cd17, github.com/apache/spark/pull/1452 + + [SPARK-2359][MLlib] Correlations + Doris Xin + 2014-07-18 17:25:32 -0700 + Commit: a243364, github.com/apache/spark/pull/1367 + + [SPARK-2571] Correctly report shuffle read metrics. + Kay Ousterhout + 2014-07-18 14:40:32 -0700 + Commit: 7b971b9, github.com/apache/spark/pull/1476 + + [SPARK-2540] [SQL] Add HiveDecimal & HiveVarchar support in unwrapping data + Cheng Hao + 2014-07-18 16:38:11 -0500 + Commit: 7f17208, github.com/apache/spark/pull/1436 + + [SPARK-2535][SQL] Add StringComparison case to NullPropagation. + Takuya UESHIN + 2014-07-18 16:24:00 -0500 + Commit: 3a1709f, github.com/apache/spark/pull/1451 + + [MLlib] SPARK-1536: multiclass classification support for decision tree + Manish Amde , manishamde , Evan Sparks + 2014-07-18 14:00:13 -0700 + Commit: d88f6be, github.com/apache/spark/pull/886 + + Reservoir sampling implementation. + Reynold Xin + 2014-07-18 12:41:50 -0700 + Commit: 586e716, github.com/apache/spark/pull/1478 + + Added t2 instance types + Basit Mustafa + 2014-07-18 12:23:47 -0700 + Commit: 7f87ab9, github.com/apache/spark/pull/1446 + + SPARK-2553. Fix compile error + Sandy Ryza + 2014-07-18 00:47:43 -0700 + Commit: 30b8d36, github.com/apache/spark/pull/1479 + + SPARK-2553. CoGroupedRDD unnecessarily allocates a Tuple2 per dependency... + Sandy Ryza + 2014-07-17 23:57:08 -0700 + Commit: e52b871, github.com/apache/spark/pull/1461 + + [SPARK-2570] [SQL] Fix the bug of ClassCastException + Cheng Hao + 2014-07-17 23:25:01 -0700 + Commit: 29809a6, github.com/apache/spark/pull/1475 + + [SPARK-2411] Add a history-not-found page to standalone Master + Andrew Or + 2014-07-17 19:45:59 -0700 + Commit: 6afca2d, github.com/apache/spark/pull/1336 + + [SPARK-2299] Consolidate various stageIdTo* hash maps in JobProgressListener + Reynold Xin + 2014-07-17 18:58:48 -0700 + Commit: 72e9021, github.com/apache/spark/pull/1262 + + SPARK-1215 [MLLIB]: Clustering: Index out of bounds error (2) + Joseph K. Bradley + 2014-07-17 15:05:02 -0700 + Commit: 935fe65, github.com/apache/spark/pull/1468 + + SPARK-1478.2 Fix incorrect NioServerSocketChannelFactory constructor call + Sean Owen + 2014-07-17 12:20:48 -0700 + Commit: 1fcd5dc, github.com/apache/spark/pull/1466 + + [SPARK-2534] Avoid pulling in the entire RDD in various operators + Reynold Xin + 2014-07-17 10:54:53 -0700 + Commit: d988d34, github.com/apache/spark/pull/1450 + + [SPARK-2423] Clean up SparkSubmit for readability + Andrew Or + 2014-07-17 01:13:32 -0700 + Commit: 9c73822, github.com/apache/spark/pull/1349 + + SPARK-2526: Simplify options in make-distribution.sh + Patrick Wendell + 2014-07-17 01:02:35 -0700 + Commit: d0ea496, github.com/apache/spark/pull/1445 + + [SPARK-2412] CoalescedRDD throws exception with certain pref locs + Aaron Davidson + 2014-07-17 01:01:14 -0700 + Commit: 7c23c0d, github.com/apache/spark/pull/1337 + + [SPARK-2154] Schedule next Driver when one completes (standalone mode) + Aaron Davidson + 2014-07-16 14:16:48 -0700 + Commit: 9c24974, github.com/apache/spark/pull/1405 + + SPARK-1097: Do not introduce deadlock while fixing concurrency bug + Aaron Davidson + 2014-07-16 14:10:17 -0700 + Commit: 8867cd0, github.com/apache/spark/pull/1409 + + [SPARK-2317] Improve task logging. + Reynold Xin + 2014-07-16 11:50:49 -0700 + Commit: 7c8d123, github.com/apache/spark/pull/1259 + + fix compile error of streaming project + James Z.M. Gao + 2014-07-16 11:35:21 -0700 + Commit: caa163f, github.com/apache/spark/pull/153 + + [SPARK-2522] set default broadcast factory to torrent + Xiangrui Meng + 2014-07-16 11:27:51 -0700 + Commit: 96f28c9, github.com/apache/spark/pull/1437 + + [SPARK-2517] Remove some compiler warnings. + Reynold Xin + 2014-07-16 11:15:07 -0700 + Commit: ef48222, github.com/apache/spark/pull/1433 + + [SPARK-2518][SQL] Fix foldability of Substring expression. + Takuya UESHIN + 2014-07-16 11:13:38 -0700 + Commit: cc965ee, github.com/apache/spark/pull/1432 + + SPARK-2519. Eliminate pattern-matching on Tuple2 in performance-critical... + Sandy Ryza + 2014-07-16 11:07:16 -0700 + Commit: fc7edc9, github.com/apache/spark/pull/1435 + + [SQL] Cleaned up ConstantFolding slightly. + Reynold Xin + 2014-07-16 10:55:47 -0700 + Commit: 1c5739f, github.com/apache/spark/pull/1430 + + [SPARK-2525][SQL] Remove as many compilation warning messages as possible in Spark SQL + Yin Huai + 2014-07-16 10:53:59 -0700 + Commit: df95d82, github.com/apache/spark/pull/1444 + + Tightening visibility for various Broadcast related classes. + Reynold Xin + 2014-07-16 10:44:54 -0700 + Commit: efe2a8b, github.com/apache/spark/pull/1438 + + SPARK-2277: make TaskScheduler track hosts on rack + Rui Li + 2014-07-16 22:53:37 +0530 + Commit: 33e64ec, github.com/apache/spark/pull/1212 + + [SPARK-2119][SQL] Improved Parquet performance when reading off S3 + Cheng Lian + 2014-07-16 12:44:51 -0400 + Commit: efc452a, github.com/apache/spark/pull/1370 + + [SPARK-2504][SQL] Fix nullability of Substring expression. + Takuya UESHIN + 2014-07-15 22:43:48 -0700 + Commit: 632fb3d, github.com/apache/spark/pull/1426 + + [SPARK-2509][SQL] Add optimization for Substring. + Takuya UESHIN + 2014-07-15 22:35:34 -0700 + Commit: 9b38b7c, github.com/apache/spark/pull/1428 + + [SPARK-2314][SQL] Override collect and take in JavaSchemaRDD, forwarding to SchemaRDD implementations. + Aaron Staple + 2014-07-15 21:35:36 -0700 + Commit: 90ca532, github.com/apache/spark/pull/1421 + + follow pep8 None should be compared using is or is not + Ken Takagiwa + 2014-07-15 21:34:05 -0700 + Commit: 563acf5, github.com/apache/spark/pull/1422 + + [SPARK-2500] Move the logInfo for registering BlockManager to BlockManagerMasterActor.register method + Henry Saputra + 2014-07-15 21:21:52 -0700 + Commit: 9c12de5, github.com/apache/spark/pull/1424 + + [SPARK-2469] Use Snappy (instead of LZF) for default shuffle compression codec + Reynold Xin + 2014-07-15 18:47:39 -0700 + Commit: 4576d80, github.com/apache/spark/pull/1415 + + [SPARK-2498] [SQL] Synchronize on a lock when using scala reflection inside data type objects. + Zongheng Yang + 2014-07-15 17:58:28 -0700 + Commit: c2048a5, github.com/apache/spark/pull/1423 + + [SQL] Attribute equality comparisons should be done by exprId. + Michael Armbrust + 2014-07-15 17:56:17 -0700 + Commit: 502f907, github.com/apache/spark/pull/1414 + + SPARK-2407: Added internal implementation of SQL SUBSTR() + William Benton + 2014-07-15 14:11:57 -0700 + Commit: 61de65b, github.com/apache/spark/pull/1359 + + [SPARK-2474][SQL] For a registered table in OverrideCatalog, the Analyzer failed to resolve references in the format of "tableName.fieldName" + Yin Huai + 2014-07-15 14:06:45 -0700 + Commit: 8af46d5, github.com/apache/spark/pull/1406 + + [SQL] Whitelist more Hive tests. + Michael Armbrust + 2014-07-15 14:04:01 -0700 + Commit: bcd0c30, github.com/apache/spark/pull/1396 + + [SPARK-2483][SQL] Fix parsing of repeated, nested data access. + Michael Armbrust + 2014-07-15 14:01:48 -0700 + Commit: 0f98ef1, github.com/apache/spark/pull/1411 + + [SPARK-2471] remove runtime scope for jets3t + Xiangrui Meng + 2014-07-15 14:00:54 -0700 + Commit: a21f9a7, github.com/apache/spark/pull/1402 + + Added LZ4 to compression codec in configuration page. + Reynold Xin + 2014-07-15 13:13:33 -0700 + Commit: e7ec815, github.com/apache/spark/pull/1417 + + SPARK-1291: Link the spark UI to RM ui in yarn-client mode + witgo + 2014-07-15 13:52:56 -0500 + Commit: 72ea56d, github.com/apache/spark/pull/1112 + + SPARK-2480: Resolve sbt warnings "NOTE: SPARK_YARN is deprecated, please use -Pyarn flag" + witgo + 2014-07-15 10:46:17 -0700 + Commit: 9dd635e, github.com/apache/spark/pull/1404 + + Reformat multi-line closure argument. + William Benton + 2014-07-15 09:13:39 -0700 + Commit: cb09e93, github.com/apache/spark/pull/1419 + + [MLLIB] [SPARK-2222] Add multiclass evaluation metrics + Alexander Ulanov , unknown , Xiangrui Meng + 2014-07-15 08:40:22 -0700 + Commit: 04b01bb, github.com/apache/spark/pull/1155 + + README update: added "for Big Data". + Reynold Xin + 2014-07-15 02:20:01 -0700 + Commit: 6555618 + + Update README.md to include a slightly more informative project description. + Reynold Xin + 2014-07-15 02:15:29 -0700 + Commit: 8f1d422 + + [SPARK-2477][MLlib] Using appendBias for adding intercept in GeneralizedLinearAlgorithm + DB Tsai + 2014-07-15 02:14:58 -0700 + Commit: 52beb20, github.com/apache/spark/pull/1410 + + [SPARK-2399] Add support for LZ4 compression. + Reynold Xin + 2014-07-15 01:46:57 -0700 + Commit: dd95aba, github.com/apache/spark/pull/1416 + + discarded exceeded completedDrivers + lianhuiwang + 2014-07-15 00:22:06 -0700 + Commit: 7446f5f, github.com/apache/spark/pull/1114 + + [SPARK-2485][SQL] Lock usage of hive client. + Michael Armbrust + 2014-07-15 00:13:51 -0700 + Commit: c7c7ac8, github.com/apache/spark/pull/1412 + + [SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS + Kousuke Saruta + 2014-07-14 23:55:39 -0700 + Commit: c6d7574, github.com/apache/spark/pull/1326 + + Add/increase severity of warning in documentation of groupBy() + Aaron Davidson + 2014-07-14 23:38:12 -0700 + Commit: a2aa7be, github.com/apache/spark/pull/1380 + + SPARK-2486: Utils.getCallSite is now resilient to bogus frames + William Benton + 2014-07-14 23:09:13 -0700 + Commit: 1f99fea, github.com/apache/spark/pull/1413 + + [SPARK-2467] Revert SparkBuild to publish-local to both .m2 and .ivy2. + Takuya UESHIN + 2014-07-14 23:06:35 -0700 + Commit: e2255e4, github.com/apache/spark/pull/1398 + + [SPARK-2446][SQL] Add BinaryType support to Parquet I/O. + Takuya UESHIN + 2014-07-14 15:42:28 -0700 + Commit: 9fe693b, github.com/apache/spark/pull/1373 + + [SPARK-1946] Submit tasks after (configured ratio) executors have been registered + li-zhihui + 2014-07-14 15:32:49 -0500 + Commit: 3dd8af7, github.com/apache/spark/pull/900 + + [SPARK-2443][SQL] Fix slow read from partitioned tables + Zongheng Yang + 2014-07-14 13:22:24 -0700 + Commit: d60b09b, github.com/apache/spark/pull/1408 + + move some test file to match src code + Daoyuan + 2014-07-14 10:40:44 -0700 + Commit: 38ccd6e, github.com/apache/spark/pull/1401 + + Made rdd.py pep8 complaint by using Autopep8 and a little manual editing. + Prashant Sharma + 2014-07-14 00:42:59 -0700 + Commit: aab5349, github.com/apache/spark/pull/1354 + + SPARK-2363. Clean MLlib's sample data files + Sean Owen + 2014-07-13 19:27:43 -0700 + Commit: 635888c, github.com/apache/spark/pull/1394 + + SPARK-2462. Make Vector.apply public. + Sandy Ryza + 2014-07-12 16:55:15 -0700 + Commit: 4c8be64, github.com/apache/spark/pull/1389 + + [SPARK-2405][SQL] Reusue same byte buffers when creating new instance of InMemoryRelation + Michael Armbrust + 2014-07-12 12:13:32 -0700 + Commit: 1a7d7cc, github.com/apache/spark/pull/1332 + + [SPARK-2441][SQL] Add more efficient distinct operator. + Michael Armbrust + 2014-07-12 12:07:27 -0700 + Commit: 7e26b57, github.com/apache/spark/pull/1366 + + [SPARK-2455] Mark (Shippable)VertexPartition serializable + Ankur Dave + 2014-07-12 12:05:34 -0700 + Commit: 7a01352, github.com/apache/spark/pull/1376 + + Use the Executor's ClassLoader in sc.objectFile(). + Daniel Darabos + 2014-07-12 00:07:42 -0700 + Commit: 2245c87, github.com/apache/spark/pull/181 + + use specialized axpy in RowMatrix for SVD + Li Pu , Xiangrui Meng , Li Pu + 2014-07-11 23:26:47 -0700 + Commit: d38887b, github.com/apache/spark/pull/1378 + + [SPARK-1969][MLlib] Online summarizer APIs for mean, variance, min, and max + DB Tsai + 2014-07-11 23:04:43 -0700 + Commit: 5596086, github.com/apache/spark/pull/955 + + [SPARK-2457] Inconsistent description in README about build option + Kousuke Saruta + 2014-07-11 21:10:26 -0700 + Commit: cbff187, github.com/apache/spark/pull/1382 + + [SPARK-2437] Rename MAVEN_PROFILES to SBT_MAVEN_PROFILES and add SBT_MAVEN_PROPERTIES + Prashant Sharma + 2014-07-11 11:52:35 -0700 + Commit: b23e9c3, github.com/apache/spark/pull/1374 + + [Minor] Remove unused val in Master + Andrew Or + 2014-07-11 00:21:16 -0700 + Commit: f4f46de, github.com/apache/spark/pull/1365 + + fix Graph partitionStrategy comment + CrazyJvm + 2014-07-11 00:02:24 -0700 + Commit: 282cca0, github.com/apache/spark/pull/1368 + + [SPARK-2358][MLLIB] Add an option to include native BLAS/LAPACK loader in the build + Xiangrui Meng + 2014-07-10 21:57:54 -0700 + Commit: 2f59ce7, github.com/apache/spark/pull/1295 + + [SPARK-2428][SQL] Add except and intersect methods to SchemaRDD. + Takuya UESHIN + 2014-07-10 19:27:24 -0700 + Commit: 10b59ba, github.com/apache/spark/pull/1355 + + [SPARK-2415] [SQL] RowWriteSupport should handle empty ArrayType correctly. + Takuya UESHIN + 2014-07-10 19:23:44 -0700 + Commit: f5abd27, github.com/apache/spark/pull/1339 + + [SPARK-2431][SQL] Refine StringComparison and related codes. + Takuya UESHIN + 2014-07-10 19:20:00 -0700 + Commit: f62c427, github.com/apache/spark/pull/1357 + + SPARK-2427: Fix Scala examples that use the wrong command line arguments index + Artjom-Metro , Artjom-Metro + 2014-07-10 16:03:30 -0700 + Commit: ae8ca4d, github.com/apache/spark/pull/1353 + + [SPARK-1341] [Streaming] Throttle BlockGenerator to limit rate of data consumption. + Issac Buenrostro + 2014-07-10 16:01:08 -0700 + Commit: 2dd6724, github.com/apache/spark/pull/945 + + [SPARK-1478].3: Upgrade FlumeInputDStream's FlumeReceiver to support FLUME-1915 + tmalaska , Tathagata Das + 2014-07-10 13:15:02 -0700 + Commit: 40a8fef, github.com/apache/spark/pull/1347 + + name ec2 instances and security groups consistently + Nicholas Chammas , nchammas + 2014-07-10 12:56:00 -0700 + Commit: 369aa84, github.com/apache/spark/pull/1344 + + HOTFIX: Minor doc update for sbt change + Patrick Wendell + 2014-07-10 11:10:43 -0700 + Commit: 88006a6 + + [SPARK-1776] Have Spark's SBT build read dependencies from Maven. + Prashant Sharma , Patrick Wendell + 2014-07-10 11:03:37 -0700 + Commit: 628932b, github.com/apache/spark/pull/772 + + SPARK-2115: Stage kill link is too close to stage details link + Masayoshi TSUZUKI + 2014-07-10 01:18:37 -0700 + Commit: c2babc0, github.com/apache/spark/pull/1350 + + Clean up SparkKMeans example's code + Raymond Liu + 2014-07-09 23:39:29 -0700 + Commit: 2b18ea9, github.com/apache/spark/pull/1352 + + HOTFIX: Remove persistently failing test in master. + Patrick Wendell + 2014-07-09 19:44:24 -0700 + Commit: 553c578 + + Revert "[HOTFIX] Synchronize on SQLContext.settings in tests." + Patrick Wendell + 2014-07-09 19:36:38 -0700 + Commit: dd22bc2 + + SPARK-2416: Allow richer reporting of unit test results + Patrick Wendell + 2014-07-09 19:26:16 -0700 + Commit: 2e0a037, github.com/apache/spark/pull/1340 + + SPARK-1782: svd for sparse matrix using ARPACK + Li Pu , Xiangrui Meng , Li Pu + 2014-07-09 12:15:08 -0700 + Commit: 1f33e1f, github.com/apache/spark/pull/964 + + [SPARK-2417][MLlib] Fix DecisionTree tests + johnnywalleye + 2014-07-09 11:06:34 -0700 + Commit: d35e3db, github.com/apache/spark/pull/1343 + + [STREAMING] SPARK-2343: Fix QueueInputDStream with oneAtATime false + Manuel Laflamme + 2014-07-09 10:45:45 -0700 + Commit: 0eb1152, github.com/apache/spark/pull/1285 + + [SPARK-2384] Add tooltips to UI. + Kay Ousterhout + 2014-07-08 22:57:21 -0700 + Commit: 339441f, github.com/apache/spark/pull/1314 + + [SPARK-2152][MLlib] fix bin offset in DecisionTree node aggregations (also resolves SPARK-2160) + johnnywalleye + 2014-07-08 19:17:26 -0700 + Commit: 1114207, github.com/apache/spark/pull/1316 + + [SPARK-2413] Upgrade junit_xml_listener to 0.5.1 + DB Tsai + 2014-07-08 17:50:36 -0700 + Commit: ac9cdc1, github.com/apache/spark/pull/1333 + + [SPARK-2392] Executors should not start their own HTTP servers + Andrew Or + 2014-07-08 17:35:31 -0700 + Commit: bf04a39, github.com/apache/spark/pull/1335 + + [SPARK-2362] Fix for newFilesOnly logic in file DStream + Gabriele Nizzoli + 2014-07-08 14:23:38 -0700 + Commit: e6f7bfc, github.com/apache/spark/pull/1077 + + [SPARK-2409] Make SQLConf thread safe. + Reynold Xin + 2014-07-08 14:00:47 -0700 + Commit: 32516f8, github.com/apache/spark/pull/1334 + + SPARK-2400 : fix spark.yarn.max.executor.failures explaination + CrazyJvm + 2014-07-08 13:55:42 -0500 + Commit: b520b64, github.com/apache/spark/pull/1282 + + [SPARK-2403] Catch all errors during serialization in DAGScheduler + Daniel Darabos + 2014-07-08 10:43:46 -0700 + Commit: c8a2313, github.com/apache/spark/pull/1329 + + [SPARK-2395][SQL] Optimize common LIKE patterns. + Michael Armbrust + 2014-07-08 10:36:18 -0700 + Commit: cc3e0a1, github.com/apache/spark/pull/1325 + + [EC2] Add default history server port to ec2 script + Andrew Or + 2014-07-08 16:49:31 +0900 + Commit: 56e009d, github.com/apache/spark/pull/1296 + + [SPARK-2391][SQL] Custom take() for LIMIT queries. + Michael Armbrust + 2014-07-08 00:41:46 -0700 + Commit: 5a40636, github.com/apache/spark/pull/1318 + + Resolve sbt warnings during build Ⅱ + witgo + 2014-07-08 00:31:42 -0700 + Commit: 3cd5029, github.com/apache/spark/pull/1153 + + Updated programming-guide.md + Rishi Verma + 2014-07-08 00:29:23 -0700 + Commit: 0128905, github.com/apache/spark/pull/1324 + + [SPARK-2235][SQL]Spark SQL basicOperator add Intersect operator + Yanjie Gao , YanjieGao <396154235@qq.com> + 2014-07-07 19:40:04 -0700 + Commit: 50561f4, github.com/apache/spark/pull/1150 + + [SPARK-2376][SQL] Selecting list values inside nested JSON objects raises java.lang.IllegalArgumentException + Yin Huai + 2014-07-07 18:37:38 -0700 + Commit: 4352a2f, github.com/apache/spark/pull/1320 + + [SPARK-2375][SQL] JSON schema inference may not resolve type conflicts correctly for a field inside an array of structs + Yin Huai + 2014-07-07 17:05:59 -0700 + Commit: f0496ee, github.com/apache/spark/pull/1308 + + [SPARK-2386] [SQL] RowWriteSupport should use the exact types to cast. + Takuya UESHIN + 2014-07-07 17:04:02 -0700 + Commit: 4deeed1, github.com/apache/spark/pull/1315 + + [SPARK-2339][SQL] SQL parser in sql-core is case sensitive, but a table alias is converted to lower case when we create Subquery + Yin Huai + 2014-07-07 17:01:44 -0700 + Commit: c0b4cf0, github.com/apache/spark/pull/1317 + + [SPARK-1977][MLLIB] register mutable BitSet in MovieLenseALS + Neville Li + 2014-07-07 15:06:14 -0700 + Commit: f7ce1b3, github.com/apache/spark/pull/1319 + + [SPARK-2327] [SQL] Fix nullabilities of Join/Generate/Aggregate. + Takuya UESHIN + 2014-07-05 11:51:48 -0700 + Commit: 9d5ecf8, github.com/apache/spark/pull/1266 + + [SPARK-2366] [SQL] Add column pruning for the right side of LeftSemi join. + Takuya UESHIN + 2014-07-05 11:48:08 -0700 + Commit: 3da8df9, github.com/apache/spark/pull/1301 + + [SPARK-2306]:BoundedPriorityQueue is private and not registered with Kry... + ankit.bhardwaj + 2014-07-04 22:06:10 -0700 + Commit: 42f3abd, github.com/apache/spark/pull/1299 + + [SPARK-2370][SQL] Decrease metadata retrieved for partitioned hive queries. + Michael Armbrust + 2014-07-04 19:15:48 -0700 + Commit: 9d006c9, github.com/apache/spark/pull/1305 + + Added SignalLogger to HistoryServer. + Reynold Xin + 2014-07-04 17:33:07 -0700 + Commit: 0db5d5a, github.com/apache/spark/pull/1300 + + HOTFIX: Clean before building docs during release. + Patrick Wendell + 2014-07-04 10:01:19 -0700 + Commit: fc71658 + + [SPARK-2234][SQL]Spark SQL basicOperators add Except operator + Yanjie Gao , YanjieGao <396154235@qq.com>, root , gaoyanjie + 2014-07-04 02:43:57 -0700 + Commit: 5dadda8, github.com/apache/spark/pull/1151 + + [SPARK-2059][SQL] Add analysis checks + Reynold Xin , Michael Armbrust + 2014-07-04 00:53:41 -0700 + Commit: b3e768e, github.com/apache/spark/pull/1265 + + Update SQLConf.scala + baishuo(白硕) + 2014-07-04 00:25:31 -0700 + Commit: 0bbe612, github.com/apache/spark/pull/1272 + + [SPARK-1199][REPL] Remove VALId and use the original import style for defined classes. + Prashant Sharma + 2014-07-04 00:05:27 -0700 + Commit: d434150, github.com/apache/spark/pull/1179 + + [SPARK-2059][SQL] Don't throw TreeNodeException in `execution.ExplainCommand` + Cheng Lian + 2014-07-03 23:41:54 -0700 + Commit: 5448804, github.com/apache/spark/pull/1294 + + SPARK-2282: Reuse PySpark Accumulator sockets to avoid crashing Spark + Aaron Davidson + 2014-07-03 23:02:36 -0700 + Commit: 97a0bfe, github.com/apache/spark/pull/1220 + + [SPARK-2307][Reprise] Correctly report RDD blocks on SparkUI + Andrew Or + 2014-07-03 22:48:23 -0700 + Commit: 3894a49, github.com/apache/spark/pull/1255 + + [SPARK-2350] Don't NPE while launching drivers + Aaron Davidson + 2014-07-03 22:31:41 -0700 + Commit: 586feb5, github.com/apache/spark/pull/1289 + + [SPARK-1097] Workaround Hadoop conf ConcurrentModification issue + Raymond Liu + 2014-07-03 19:24:22 -0700 + Commit: 5fa0a05, github.com/apache/spark/pull/1273 + + Streaming programming guide typos + Clément MATHIEU + 2014-07-03 18:31:18 -0700 + Commit: fdc4c11, github.com/apache/spark/pull/1286 + + [HOTFIX] Synchronize on SQLContext.settings in tests. + Zongheng Yang + 2014-07-03 17:37:53 -0700 + Commit: d4c30cd, github.com/apache/spark/pull/1277 + + [SPARK-2109] Setting SPARK_MEM for bin/pyspark does not work. + Prashant Sharma + 2014-07-03 15:06:58 -0700 + Commit: 731f683b, github.com/apache/spark/pull/1050 + + [SPARK-2342] Evaluation helper's output type doesn't conform to input ty... + Yijie Shen + 2014-07-03 13:22:13 -0700 + Commit: a9b52e5, github.com/apache/spark/pull/1283 + + SPARK-1675. Make clear whether computePrincipalComponents requires centered data + Sean Owen + 2014-07-03 11:54:51 -0700 + Commit: 2b36344, github.com/apache/spark/pull/1171 + + [SPARK] Fix NPE for ExternalAppendOnlyMap + Andrew Or + 2014-07-03 10:26:50 -0700 + Commit: c480537, github.com/apache/spark/pull/1288 + + [SPARK-2324] SparkContext should not exit directly when spark.local.dir is a list of multiple paths and one of them has error + yantangzhai + 2014-07-03 10:14:35 -0700 + Commit: 3bbeca6, github.com/apache/spark/pull/1274 + + [SPARK-2287] [SQL] Make ScalaReflection be able to handle Generic case classes. + Takuya UESHIN + 2014-07-02 10:10:36 -0700 + Commit: bc7041a, github.com/apache/spark/pull/1226 + + [SPARK-2328] [SQL] Add execution of `SHOW TABLES` before `TestHive.reset()`. + Takuya UESHIN + 2014-07-02 10:07:01 -0700 + Commit: 1e2c26c, github.com/apache/spark/pull/1268 + + SPARK-2186: Spark SQL DSL support for simple aggregations such as SUM and AVG + Ximo Guanter Gonzalbez + 2014-07-02 10:03:44 -0700 + Commit: 5c6ec94, github.com/apache/spark/pull/1211 + + update the comments in SqlParser + CodingCat + 2014-07-01 20:37:10 -0700 + Commit: 6596392, github.com/apache/spark/pull/1275 + + [SPARK-2185] Emit warning when task size exceeds a threshold. + Kay Ousterhout + 2014-07-01 01:56:51 -0700 + Commit: 05c3d90, github.com/apache/spark/pull/1149 + + SPARK-2332 [build] add exclusion for old servlet-api on hadoop-client in core + Peter MacKinnon + 2014-07-01 00:28:38 -0700 + Commit: 3319a3e, github.com/apache/spark/pull/1271 + + SPARK-2293. Replace RDD.zip usage by map with predict inside. + Sean Owen + 2014-06-30 16:03:38 -0700 + Commit: 04fa122, github.com/apache/spark/pull/1250 + + [SPARK-2318] When exiting on a signal, print the signal name first. + Reynold Xin + 2014-06-30 15:12:38 -0700 + Commit: 5fccb56, github.com/apache/spark/pull/1260 + + [SPARK-2322] Exception in resultHandler should NOT crash DAGScheduler and shutdown SparkContext. + Reynold Xin + 2014-06-30 11:50:22 -0700 + Commit: 358ae15, github.com/apache/spark/pull/1264 + + SPARK-2077 Log serializer that actually ends up being used + Andrew Ash + 2014-06-29 23:29:05 -0700 + Commit: 6803642, github.com/apache/spark/pull/1017 + + SPARK-897: preemptively serialize closures + William Benton + 2014-06-29 23:27:34 -0700 + Commit: a484030, github.com/apache/spark/pull/143 + + [SPARK-2104] Fix task serializing issues when sort with Java non serializable class + jerryshao + 2014-06-29 23:00:00 -0700 + Commit: 66135a3, github.com/apache/spark/pull/1245 + + [SPARK-1683] Track task read metrics. + Kay Ousterhout + 2014-06-29 22:01:42 -0700 + Commit: 7b71a0e, github.com/apache/spark/pull/962 + + [SPARK-2320] Reduce exception/code block font size in web ui + Reynold Xin + 2014-06-29 16:46:28 -0700 + Commit: cdf613f, github.com/apache/spark/pull/1261 + + Improve MapOutputTracker error logging. + Reynold Xin + 2014-06-28 21:05:03 -0700 + Commit: 2053d79, github.com/apache/spark/pull/1258 + + [SPARK-1394] Remove SIGCHLD handler in worker subprocess + Matthew Farrellee + 2014-06-28 18:39:27 -0700 + Commit: 3c104c7, github.com/apache/spark/pull/1247 + + [SPARK-2233] make-distribution script should list the git hash in the RELEASE file + Guillaume Ballet + 2014-06-28 13:07:12 -0700 + Commit: b8f2e13, github.com/apache/spark/pull/1216 + + [SPARK-2003] Fix python SparkContext example + Matthew Farrellee + 2014-06-27 18:20:33 -0700 + Commit: 0e0686d, github.com/apache/spark/pull/1246 + + [SPARK-2259] Fix highly misleading docs on cluster / client deploy modes + Andrew Or + 2014-06-27 16:11:31 -0700 + Commit: f17510e, github.com/apache/spark/pull/1200 + + [SPARK-2307] SparkUI - storage tab displays incorrect RDDs + Andrew Or + 2014-06-27 15:23:25 -0700 + Commit: 21e0f77, github.com/apache/spark/pull/1249 + + SPARK-2181:The keys for sorting the columns of Executor page in SparkUI are incorrect + witgo + 2014-06-26 21:59:21 -0700 + Commit: 18f29b9, github.com/apache/spark/pull/1135 + + [SPARK-2251] fix concurrency issues in random sampler + Xiangrui Meng + 2014-06-26 21:46:55 -0700 + Commit: c23f5db, github.com/apache/spark/pull/1229 + + [SPARK-2297][UI] Make task attempt and speculation more explicit in UI. + Reynold Xin + 2014-06-26 21:13:26 -0700 + Commit: d1636dd, github.com/apache/spark/pull/1236 + + Removed throwable field from FetchFailedException and added MetadataFetchFailedException + Reynold Xin + 2014-06-26 21:12:16 -0700 + Commit: bf578de, github.com/apache/spark/pull/1227 + + [SQL]Extract the joinkeys from join condition + Cheng Hao + 2014-06-26 19:18:11 -0700 + Commit: 981bde9, github.com/apache/spark/pull/1190 + + Strip '@' symbols when merging pull requests. + Patrick Wendell + 2014-06-26 17:09:24 -0700 + Commit: f1f7385, github.com/apache/spark/pull/1239 + + Fixing AWS instance type information based upon current EC2 data + Zichuan Ye + 2014-06-26 15:21:29 -0700 + Commit: 62d4a0f, github.com/apache/spark/pull/1156 + + [SPARK-2286][UI] Report exception/errors for failed tasks that are not ExceptionFailure + Reynold Xin + 2014-06-26 14:00:45 -0700 + Commit: 6587ef7, github.com/apache/spark/pull/1225 + + [SPARK-2295] [SQL] Make JavaBeans nullability stricter. + Takuya UESHIN + 2014-06-26 13:37:19 -0700 + Commit: 32a1ad7, github.com/apache/spark/pull/1235 + + Remove use of spark.worker.instances + Kay Ousterhout + 2014-06-26 08:20:27 -0500 + Commit: 48a82a8, github.com/apache/spark/pull/1214 + + [SPARK-2254] [SQL] ScalaRefection should mark primitive types as non-nullable. + Takuya UESHIN + 2014-06-25 23:55:31 -0700 + Commit: e4899a2, github.com/apache/spark/pull/1193 + + [SPARK-2172] PySpark cannot import mllib modules in YARN-client mode + Szul, Piotr + 2014-06-25 21:55:49 -0700 + Commit: 441cdcc, github.com/apache/spark/pull/1223 + + [SPARK-2284][UI] Mark all failed tasks as failures. + Reynold Xin + 2014-06-25 22:35:03 -0700 + Commit: 4a346e2, github.com/apache/spark/pull/1224 + + [SPARK-1749] Job cancellation when SchedulerBackend does not implement killTask + Mark Hamstra , Kay Ousterhout + 2014-06-25 20:57:48 -0700 + Commit: b88a59a, github.com/apache/spark/pull/1219 + + [SPARK-2283][SQL] Reset test environment before running PruningSuite + Cheng Lian + 2014-06-25 18:41:47 -0700 + Commit: 7f196b0, github.com/apache/spark/pull/1221 + + [SQL] SPARK-1800 Add broadcast hash join operator & associated hints. + Zongheng Yang , Michael Armbrust + 2014-06-25 18:06:33 -0700 + Commit: 9d824fe, github.com/apache/spark/pull/1163 + + [SPARK-2204] Launch tasks on the proper executors in mesos fine-grained mode + Sebastien Rainville + 2014-06-25 13:21:18 -0700 + Commit: 1132e47, github.com/apache/spark/pull/1140 + + [SPARK-2270] Kryo cannot serialize results returned by asJavaIterable + Reynold Xin + 2014-06-25 12:43:22 -0700 + Commit: 7ff2c75, github.com/apache/spark/pull/1206 + + [SPARK-2258 / 2266] Fix a few worker UI bugs + Andrew Or + 2014-06-25 12:23:08 -0700 + Commit: 9aa6032, github.com/apache/spark/pull/1213 + + [SPARK-2242] HOTFIX: pyspark shell hangs on simple job + Andrew Or + 2014-06-25 10:47:22 -0700 + Commit: 5603e4c, github.com/apache/spark/pull/1178 + + Replace doc reference to Shark with Spark SQL. + Reynold Xin + 2014-06-25 01:01:23 -0700 + Commit: ac06a85 + + SPARK-2038: rename "conf" parameters in the saveAsHadoop functions with source-compatibility + CodingCat + 2014-06-25 00:23:32 -0700 + Commit: acc01ab, github.com/apache/spark/pull/1137 + + [BUGFIX][SQL] Should match java.math.BigDecimal when wnrapping Hive output + Cheng Lian + 2014-06-25 00:17:28 -0700 + Commit: 22036ae, github.com/apache/spark/pull/1199 + + [SPARK-2263][SQL] Support inserting MAP to Hive tables + Cheng Lian + 2014-06-25 00:14:34 -0700 + Commit: 8fade89, github.com/apache/spark/pull/1205 + + SPARK-2248: spark.default.parallelism does not apply in local mode + witgo + 2014-06-24 19:44:37 -0700 + Commit: b6b4485, github.com/apache/spark/pull/1194 + + Fix possible null pointer in acumulator toString + Michael Armbrust + 2014-06-24 19:39:19 -0700 + Commit: 2714968, github.com/apache/spark/pull/1204 + + Autodetect JAVA_HOME on RPM-based systems + Matthew Farrellee + 2014-06-24 19:31:20 -0700 + Commit: 54055fb, github.com/apache/spark/pull/1185 + + [SQL]Add base row updating methods for JoinedRow + Cheng Hao + 2014-06-24 19:07:02 -0700 + Commit: 133495d, github.com/apache/spark/pull/1187 + + [SPARK-1112, 2156] Bootstrap to fetch the driver's Spark properties. + Xiangrui Meng + 2014-06-24 19:06:07 -0700 + Commit: 8ca4176, github.com/apache/spark/pull/1132 + + [SPARK-2264][SQL] Fix failing CachedTableSuite + Michael Armbrust + 2014-06-24 19:04:29 -0700 + Commit: a162c9b, github.com/apache/spark/pull/1201 + + Fix broken Json tests. + Kay Ousterhout + 2014-06-24 16:54:50 -0700 + Commit: 1978a90, github.com/apache/spark/pull/1198 + + HOTFIX: Disabling tests per SPARK-2264 + Patrick Wendell + 2014-06-24 15:09:30 -0700 + Commit: 221909e + + SPARK-1937: fix issue with task locality + Rui Li , lirui-intel + 2014-06-24 11:40:37 -0700 + Commit: 924b708, github.com/apache/spark/pull/892 + + [SPARK-2252] Fix MathJax for HTTPs. + Reynold Xin + 2014-06-23 23:18:47 -0700 + Commit: 420c1c3, github.com/apache/spark/pull/1189 + + [SPARK-2124] Move aggregation into shuffle implementations + jerryshao + 2014-06-23 20:25:46 -0700 + Commit: 56eb8af, github.com/apache/spark/pull/1064 + + [SPARK-2227] Support dfs command in SQL. + Reynold Xin + 2014-06-23 18:34:54 -0700 + Commit: 51c8168, github.com/apache/spark/pull/1167 + + Cleanup on Connection, ConnectionManagerId, ConnectionManager classes part 2 + Henry Saputra + 2014-06-23 17:13:26 -0700 + Commit: 383bf72, github.com/apache/spark/pull/1157 + + [SPARK-1768] History server enhancements. + Marcelo Vanzin + 2014-06-23 13:53:44 -0700 + Commit: 21ddd7d, github.com/apache/spark/pull/718 + + [SPARK-2118] spark class should complain if tools jar is missing. + Prashant Sharma + 2014-06-23 13:35:09 -0700 + Commit: 6dc6722, github.com/apache/spark/pull/1068 + + [SPARK-1669][SQL] Made cacheTable idempotent + Cheng Lian + 2014-06-23 13:24:33 -0700 + Commit: a4bc442, github.com/apache/spark/pull/1183 + + Fix mvn detection + Matthew Farrellee + 2014-06-23 11:24:05 -0700 + Commit: 853a2b9, github.com/apache/spark/pull/1181 + + Fixed small running on YARN docs typo + Vlad + 2014-06-23 10:55:49 -0500 + Commit: b88238f, github.com/apache/spark/pull/1158 + + [SPARK-1395] Fix "local:" URI support in Yarn mode (again). + Marcelo Vanzin + 2014-06-23 08:51:11 -0500 + Commit: e380767, github.com/apache/spark/pull/560 + + SPARK-2166 - Listing of instances to be terminated before the prompt + Jean-Martin Archer + 2014-06-22 20:52:02 -0700 + Commit: 9cb64b2, github.com/apache/spark/pull/270 + + SPARK-2241: quote command line args in ec2 script + Ori Kremer + 2014-06-22 20:21:23 -0700 + Commit: 9fc373e, github.com/apache/spark/pull/1169 + + SPARK-2229: FileAppender throw an llegalArgumentException in jdk6 + witgo + 2014-06-22 18:25:16 -0700 + Commit: 409d24e, github.com/apache/spark/pull/1174 + + SPARK-1316. Remove use of Commons IO + Sean Owen + 2014-06-22 11:47:49 -0700 + Commit: 9fe28c3, github.com/apache/spark/pull/1173 + + SPARK-2034. KafkaInputDStream doesn't close resources and may prevent JVM shutdown + Sean Owen + 2014-06-22 01:12:15 -0700 + Commit: 476581e, github.com/apache/spark/pull/980 + + SPARK-2231: dev/run-tests should include YARN and use a recent Hadoop version + Patrick Wendell + 2014-06-22 00:55:27 -0700 + Commit: 58b32f3, github.com/apache/spark/pull/1175 + + SPARK-1996. Remove use of special Maven repo for Akka + Sean Owen + 2014-06-21 23:29:57 -0700 + Commit: 1db9cbc, github.com/apache/spark/pull/1170 + + HOTFIX: Add excludes for new MIMA files + Patrick Wendell + 2014-06-21 15:20:15 -0700 + Commit: 3e0b078 + + HOTFIX: Fix missing MIMA ignore + Patrick Wendell + 2014-06-21 13:02:49 -0700 + Commit: 0a432d6 + + [SQL] Break hiveOperators.scala into multiple files. + Reynold Xin + 2014-06-21 12:04:18 -0700 + Commit: ec935ab, github.com/apache/spark/pull/1166 + + [SQL] Pass SQLContext instead of SparkContext into physical operators. + Reynold Xin + 2014-06-20 22:49:48 -0700 + Commit: ca5d8b5, github.com/apache/spark/pull/1164 + + Fix some tests. + Marcelo Vanzin + 2014-06-20 20:05:12 -0700 + Commit: 648553d, github.com/apache/spark/pull/917 + + [SPARK-2061] Made splits deprecated in JavaRDDLike + Anant + 2014-06-20 18:54:00 -0700 + Commit: 010c460, github.com/apache/spark/pull/1062 + + HOTFIX: Fixing style error introduced by 08d0ac + Patrick Wendell + 2014-06-20 18:44:54 -0700 + Commit: a678642 + + [SPARK-1970] Update unit test in XORShiftRandomSuite to use ChiSquareTest from commons-math3 + Doris Xin + 2014-06-20 18:42:02 -0700 + Commit: e99903b, github.com/apache/spark/pull/1073 + + SPARK-1902 Silence stacktrace from logs when doing port failover to port n+1 + Andrew Ash + 2014-06-20 18:25:33 -0700 + Commit: 08d0aca, github.com/apache/spark/pull/1019 + + [SQL] Use hive.SessionState, not the thread local SessionState + Aaron Davidson + 2014-06-20 17:55:54 -0700 + Commit: 2044784, github.com/apache/spark/pull/1148 + + Move ScriptTransformation into the appropriate place. + Reynold Xin + 2014-06-20 17:16:56 -0700 + Commit: d4c7572, github.com/apache/spark/pull/1162 + + Clean up CacheManager et al. + Andrew Or + 2014-06-20 17:14:33 -0700 + Commit: 01125a1, github.com/apache/spark/pull/1083 + + [SPARK-2225] Turn HAVING without GROUP BY into WHERE. + Reynold Xin + 2014-06-20 15:38:02 -0700 + Commit: 0ac71d1, github.com/apache/spark/pull/1161 + + SPARK-2180: support HAVING clauses in Hive queries + William Benton + 2014-06-20 13:41:38 -0700 + Commit: 171ebb3, github.com/apache/spark/pull/1136 + + SPARK-1868: Users should be allowed to cogroup at least 4 RDDs + Allan Douglas R. de Oliveira + 2014-06-20 11:03:03 -0700 + Commit: 6a224c3, github.com/apache/spark/pull/813 + + [SPARK-2163] class LBFGS optimize with Double tolerance instead of Int + Gang Bai + 2014-06-20 08:52:20 -0700 + Commit: d484dde, github.com/apache/spark/pull/1104 + + [SPARK-2218] rename Equals to EqualTo in Spark SQL expressions. + Reynold Xin + 2014-06-20 00:34:59 -0700 + Commit: 2f6a835, github.com/apache/spark/pull/1146 + + [SPARK-2196] [SQL] Fix nullability of CaseWhen. + Takuya UESHIN + 2014-06-20 00:12:52 -0700 + Commit: 3249528, github.com/apache/spark/pull/1133 + + SPARK-2203: PySpark defaults to use same num reduce partitions as map side + Aaron Davidson + 2014-06-20 00:06:57 -0700 + Commit: f46e02f, github.com/apache/spark/pull/1138 + + [SPARK-2209][SQL] Cast shouldn't do null check twice. + Reynold Xin + 2014-06-20 00:01:19 -0700 + Commit: c55bbb4, github.com/apache/spark/pull/1143 + + [SPARK-2210] cast to boolean on boolean value gets turned into NOT((boolean_condition) = 0) + Reynold Xin + 2014-06-19 23:58:23 -0700 + Commit: 6175640, github.com/apache/spark/pull/1144 + + SPARK-1293 [SQL] Parquet support for nested types + Andre Schumacher , Michael Armbrust + 2014-06-19 23:47:45 -0700 + Commit: f479cf3, github.com/apache/spark/pull/360 + + [SPARK-2177][SQL] describe table result contains only one column + Yin Huai + 2014-06-19 23:41:38 -0700 + Commit: f397e92, github.com/apache/spark/pull/1118 + + [SQL] Improve Speed of InsertIntoHiveTable + Michael Armbrust + 2014-06-19 23:39:03 -0700 + Commit: d3b7671, github.com/apache/spark/pull/1130 + + More minor scaladoc cleanup for Spark SQL. + Reynold Xin + 2014-06-19 22:34:21 -0700 + Commit: 278ec8a, github.com/apache/spark/pull/1142 + + HOTFIX: SPARK-2208 local metrics tests can fail on fast machines + Patrick Wendell + 2014-06-19 21:06:28 -0700 + Commit: e551479, github.com/apache/spark/pull/1141 + + A few minor Spark SQL Scaladoc fixes. + Reynold Xin + 2014-06-19 18:24:05 -0700 + Commit: 5464e79, github.com/apache/spark/pull/1139 + + [SPARK-2151] Recognize memory format for spark-submit + nravi + 2014-06-19 17:11:06 -0700 + Commit: f14b00a, github.com/apache/spark/pull/1095 + + [SPARK-2191][SQL] Make sure InsertIntoHiveTable doesn't execute more than once. + Michael Armbrust + 2014-06-19 14:14:03 -0700 + Commit: 777c595, github.com/apache/spark/pull/1129 + + [SPARK-2051]In yarn.ClientBase spark.yarn.dist.* do not work + witgo + 2014-06-19 12:11:26 -0500 + Commit: bce0897, github.com/apache/spark/pull/969 + + Minor fix + WangTao + 2014-06-18 23:24:57 -0700 + Commit: 67fca18, github.com/apache/spark/pull/1105 + + [SPARK-2187] Explain should not run the optimizer twice. + Reynold Xin + 2014-06-18 22:44:12 -0700 + Commit: 640c294, github.com/apache/spark/pull/1123 + + Squishing a typo bug before it causes real harm + Doris Xin + 2014-06-18 22:19:06 -0700 + Commit: 566f70f, github.com/apache/spark/pull/1125 + + [SPARK-2184][SQL] AddExchange isn't idempotent + Michael Armbrust + 2014-06-18 17:52:42 -0700 + Commit: 5ff75c7, github.com/apache/spark/pull/1122 + + Remove unicode operator from RDD.scala + Doris Xin + 2014-06-18 15:01:29 -0700 + Commit: 45a95f8, github.com/apache/spark/pull/1119 + + SPARK-2158 Clean up core/stdout file from FileAppenderSuite + Mark Hamstra + 2014-06-18 14:56:41 -0700 + Commit: 4cbeea8, github.com/apache/spark/pull/1100 + + [SPARK-1466] Raise exception if pyspark Gateway process doesn't start. + Kay Ousterhout + 2014-06-18 13:16:26 -0700 + Commit: 3870248, github.com/apache/spark/pull/383 + + Updated the comment for SPARK-2162. + Reynold Xin + 2014-06-18 12:48:58 -0700 + Commit: dd96fcd, github.com/apache/spark/pull/1117 + + [SPARK-2162] Double check in doGetLocal to avoid read on removed block. + Raymond Liu + 2014-06-18 10:57:45 -0700 + Commit: 5ad5e34, github.com/apache/spark/pull/1103 + + [SPARK-2176][SQL] Extra unnecessary exchange operator in the result of an explain command + Yin Huai + 2014-06-18 10:51:32 -0700 + Commit: 587d320, github.com/apache/spark/pull/1116 + + [STREAMING] SPARK-2009 Key not found exception when slow receiver starts + Vadim Chekan + 2014-06-17 22:03:50 -0700 + Commit: 889f7b7, github.com/apache/spark/pull/961 + + Revert "SPARK-2038: rename "conf" parameters in the saveAsHadoop functions" + Patrick Wendell + 2014-06-17 19:34:17 -0700 + Commit: 9e4b4bd + + [SPARK-2060][SQL] Querying JSON Datasets with SQL and DSL in Spark SQL + Yin Huai + 2014-06-17 19:14:59 -0700 + Commit: d2f4f30, github.com/apache/spark/pull/999 + + HOTFIX: bug caused by #941 + Patrick Wendell + 2014-06-17 15:09:24 -0700 + Commit: b2ebf42, github.com/apache/spark/pull/1108 + + [SPARK-2147 / 2161] Show removed executors on the UI + Andrew Or + 2014-06-17 12:25:55 -0700 + Commit: a14807e, github.com/apache/spark/pull/1102 + + SPARK-2038: rename "conf" parameters in the saveAsHadoop functions + CodingCat + 2014-06-17 12:17:48 -0700 + Commit: 443f5e1, github.com/apache/spark/pull/1087 + + SPARK-2146. Fix takeOrdered doc + Sandy Ryza + 2014-06-17 12:03:22 -0700 + Commit: 2794990, github.com/apache/spark/pull/1086 + + SPARK-1063 Add .sortBy(f) method on RDD + Andrew Ash + 2014-06-17 11:47:48 -0700 + Commit: b92d16b, github.com/apache/spark/pull/369 + + [SPARK-2053][SQL] Add Catalyst expressions for CASE WHEN. + Zongheng Yang + 2014-06-17 13:30:17 +0200 + Commit: e243c5f, github.com/apache/spark/pull/1055 + + [SPARK-2164][SQL] Allow Hive UDF on columns of type struct + Xi Liu + 2014-06-17 13:14:40 +0200 + Commit: f5a4049, github.com/apache/spark/pull/796 + + [SPARK-2144] ExecutorsPage reports incorrect # of RDD blocks + Andrew Or + 2014-06-17 01:28:22 -0700 + Commit: 09deb3e, github.com/apache/spark/pull/1080 + + SPARK-2035: Store call stack for stages, display it on the UI. + Daniel Darabos , Patrick Wendell + 2014-06-17 00:08:05 -0700 + Commit: 23a12ce, github.com/apache/spark/pull/981 + + SPARK-1990: added compatibility for python 2.6 for ssh_read command + Anant + 2014-06-16 23:42:27 -0700 + Commit: 8cd04c3, github.com/apache/spark/pull/941 + + [SPARK-2130] End-user friendly String repr for StorageLevel in Python + Kan Zhang + 2014-06-16 23:31:31 -0700 + Commit: d81c08b, github.com/apache/spark/pull/1096 + + MLlib documentation fix + Anatoli Fomenko + 2014-06-16 23:10:36 -0700 + Commit: 7afa912, github.com/apache/spark/pull/1098 + + Minor fix: made "EXPLAIN" output to play well with JDBC output format + Cheng Lian + 2014-06-16 16:42:17 -0700 + Commit: 237b96b, github.com/apache/spark/pull/1097 + + [SQL][SPARK-2094] Follow up of PR #1071 for Java API + Cheng Lian + 2014-06-16 21:30:29 +0200 + Commit: 273afcb, github.com/apache/spark/pull/1085 + + [SPARK-1930] The Container is running beyond physical memory limits, so as to be killed + witgo + 2014-06-16 14:27:31 -0500 + Commit: cdf2b04, github.com/apache/spark/pull/894 + + [SPARK-2010] Support for nested data in PySpark SQL + Kan Zhang + 2014-06-16 11:11:29 -0700 + Commit: 4fdb491, github.com/apache/spark/pull/1041 + + SPARK-2039: apply output dir existence checking for all output formats + CodingCat + 2014-06-15 23:47:58 -0700 + Commit: 716c88a, github.com/apache/spark/pull/1088 + + Updating docs to include missing information about reducers and clarify ... + Ali Ghodsi + 2014-06-15 23:44:30 -0700 + Commit: 119b06a, github.com/apache/spark/pull/1089 + + SPARK-2148 Add link to requirements for custom equals() and hashcode() methods + Andrew Ash + 2014-06-15 23:32:55 -0700 + Commit: 9672ee0, github.com/apache/spark/pull/1092 + + SPARK-1999: StorageLevel in storage tab and RDD Storage Info never changes + CrazyJvm + 2014-06-15 23:23:26 -0700 + Commit: a63aa1a, github.com/apache/spark/pull/968 + + [SPARK-937] adding EXITED executor state and not relaunching cleanly exited executors + Kan Zhang + 2014-06-15 14:55:34 -0700 + Commit: ca5d9d4, github.com/apache/spark/pull/306 + + [SQL] Support transforming TreeNodes with Option children. + Michael Armbrust , Zongheng Yang + 2014-06-15 11:28:34 +0200 + Commit: 269fc62, github.com/apache/spark/pull/1074 + + [SPARK-1837] NumericRange should be partitioned in the same way as other... + Kan Zhang + 2014-06-14 14:31:28 -0700 + Commit: 7dd9fc6, github.com/apache/spark/pull/776 + + [SPARK-2013] Documentation for saveAsPickleFile and pickleFile in Python + Kan Zhang + 2014-06-14 13:22:30 -0700 + Commit: b52603b, github.com/apache/spark/pull/983 + + [SPARK-2079] Support batching when serializing SchemaRDD to Python + Kan Zhang + 2014-06-14 13:17:22 -0700 + Commit: 2550533, github.com/apache/spark/pull/1023 + + [Spark-2137][SQL] Timestamp UDFs broken + Yin Huai + 2014-06-13 23:28:57 -0700 + Commit: 8919685, github.com/apache/spark/pull/1081 + + Small correction in Streaming Programming Guide doc + akkomar + 2014-06-13 15:37:26 -0700 + Commit: edb1f0e, github.com/apache/spark/pull/1079 + + [SPARK-2094][SQL] "Exactly once" semantics for DDL and command statements + Cheng Lian + 2014-06-13 12:59:48 -0700 + Commit: ac96d96, github.com/apache/spark/pull/1071 + + [SPARK-1964][SQL] Add timestamp to HiveMetastoreTypes.toMetastoreType + Michael Armbrust + 2014-06-13 12:55:15 -0700 + Commit: 1c2fd01, github.com/apache/spark/pull/1061 + + Workaround in Spark for ConcurrentModification issue (JIRA Hadoop-10456, Spark-1097) + nravi + 2014-06-13 10:52:21 -0700 + Commit: 70c8116, github.com/apache/spark/pull/1000 + + [HOTFIX] add math3 version to pom + Xiangrui Meng + 2014-06-13 02:59:38 -0700 + Commit: b3736e3, github.com/apache/spark/pull/1075 + + [SPARK-2135][SQL] Use planner for in-memory scans + Michael Armbrust + 2014-06-12 23:09:41 -0700 + Commit: 13f8cfd, github.com/apache/spark/pull/1072 + + [SPARK-1516]Throw exception in yarn client instead of run system.exit directly. + John Zhao + 2014-06-12 21:39:00 -0700 + Commit: f95ac68, github.com/apache/spark/pull/490 + + [Minor] Fix style, formatting and naming in BlockManager etc. + Andrew Or + 2014-06-12 20:40:58 -0700 + Commit: 44daec5, github.com/apache/spark/pull/1058 + + SPARK-1939 Refactor takeSample method in RDD to use ScaSRS + Doris Xin , dorx , Xiangrui Meng + 2014-06-12 19:44:27 -0700 + Commit: 1de1d70, github.com/apache/spark/pull/916 + + document laziness of parallelize + Ariel Rabkin + 2014-06-12 17:51:33 -0700 + Commit: 0154587, github.com/apache/spark/pull/1070 + + SPARK-2085: [MLlib] Apply user-specific regularization instead of uniform regularization in ALS + Shuo Xiang + 2014-06-12 17:37:06 -0700 + Commit: a6e0afd, github.com/apache/spark/pull/1026 + + SPARK-1843: Replace assemble-deps with env variable. + Patrick Wendell + 2014-06-12 15:43:32 -0700 + Commit: 1c04652, github.com/apache/spark/pull/877 + + [SPARK-2080] Yarn: report HS URL in client mode, correct user in cluster mode. + Marcelo Vanzin + 2014-06-12 16:19:36 -0500 + Commit: ecde5b8, github.com/apache/spark/pull/1002 + + [SPARK-2088] fix NPE in toString + Doris Xin + 2014-06-12 12:53:07 -0700 + Commit: 83c226d, github.com/apache/spark/pull/1028 + + SPARK-554. Add aggregateByKey. + Sandy Ryza + 2014-06-12 08:14:25 -0700 + Commit: ce92a9c, github.com/apache/spark/pull/705 + + fixed typo in docstring for min() + Jeff Thompson + 2014-06-12 08:10:51 -0700 + Commit: 43d53d5, github.com/apache/spark/pull/1065 + + Cleanup on Connection and ConnectionManager + Henry Saputra + 2014-06-11 23:17:51 -0700 + Commit: 4d8ae70, github.com/apache/spark/pull/1060 + + 'killFuture' is never used + Yadong + 2014-06-11 20:58:39 -0700 + Commit: e056320, github.com/apache/spark/pull/1052 + + [SPARK-2044] Pluggable interface for shuffles + Matei Zaharia + 2014-06-11 20:45:29 -0700 + Commit: 508fd37, github.com/apache/spark/pull/1009 + + [SPARK-1672][MLLIB] Separate user and product partitioning in ALS + Tor Myklebust , Xiangrui Meng + 2014-06-11 18:16:33 -0700 + Commit: d920335, github.com/apache/spark/pull/1014 + + [SPARK-2052] [SQL] Add optimization for CaseConversionExpression's. + Takuya UESHIN + 2014-06-11 17:58:35 -0700 + Commit: 9a2448d, github.com/apache/spark/pull/990 + + HOTFIX: Forgot to remove false change in previous commit + Patrick Wendell + 2014-06-11 15:55:41 -0700 + Commit: d45e0c6 + + HOTFIX: PySpark tests should be order insensitive. + Patrick Wendell + 2014-06-11 15:54:41 -0700 + Commit: 14e6dc9, github.com/apache/spark/pull/1054 + + HOTFIX: A few PySpark tests were not actually run + Andrew Or + 2014-06-11 12:11:46 -0700 + Commit: fe78b8b, github.com/apache/spark/pull/1053 + + [SQL] Code Cleanup: Left Semi Hash Join + Daoyuan + 2014-06-11 12:08:28 -0700 + Commit: ce6deb1, github.com/apache/spark/pull/1049 + + [SPARK-2042] Prevent unnecessary shuffle triggered by take() + Sameer Agarwal + 2014-06-11 12:01:04 -0700 + Commit: 4107cce, github.com/apache/spark/pull/1048 + + SPARK-2113: awaitTermination() after stop() will hang in Spark Stremaing + Lars Albertsson + 2014-06-11 10:54:42 -0700 + Commit: 4d5c12a, github.com/apache/spark/pull/1001 + + [SPARK-2108] Mark SparkContext methods that return block information as developer API's + Prashant Sharma + 2014-06-11 10:49:34 -0700 + Commit: e508f59, github.com/apache/spark/pull/1047 + + [SPARK-2069] MIMA false positives + Prashant Sharma + 2014-06-11 10:47:06 -0700 + Commit: 5b754b4, github.com/apache/spark/pull/1021 + + SPARK-1639. Tidy up some Spark on YARN code + Sandy Ryza + 2014-06-11 07:57:28 -0500 + Commit: 2a4225d, github.com/apache/spark/pull/561 + + SPARK-2107: FilterPushdownSuite doesn't need Junit jar. + Qiuzhuang.Lian + 2014-06-11 00:36:06 -0700 + Commit: 6e11930, github.com/apache/spark/pull/1046 + + [SPARK-2091][MLLIB] use numpy.dot instead of ndarray.dot + Xiangrui Meng + 2014-06-11 00:22:40 -0700 + Commit: 0f1dc3a, github.com/apache/spark/pull/1035 + + [SPARK-1968][SQL] SQL/HiveQL command for caching/uncaching tables + Cheng Lian + 2014-06-11 00:06:50 -0700 + Commit: 0266a0c, github.com/apache/spark/pull/1038 + + [SPARK-2093] [SQL] NullPropagation should use exact type value. + Takuya UESHIN + 2014-06-10 23:13:48 -0700 + Commit: 0402bd7, github.com/apache/spark/pull/1034 + + HOTFIX: clear() configs in SQLConf-related unit tests. + Zongheng Yang + 2014-06-10 21:59:01 -0700 + Commit: 601032f, github.com/apache/spark/pull/1040 + + [SPARK-2065] give launched instances names + Nicholas Chammas , nchammas + 2014-06-10 21:49:08 -0700 + Commit: a2052a4, github.com/apache/spark/pull/1043 + + Resolve scalatest warnings during build + witgo + 2014-06-10 20:24:05 -0700 + Commit: c48b622, github.com/apache/spark/pull/1032 + + [SPARK-1940] Enabling rolling of executor logs, and automatic cleanup of old executor logs + Tathagata Das + 2014-06-10 20:22:02 -0700 + Commit: 4823bf4, github.com/apache/spark/pull/895 + + [SPARK-1998] SparkFlumeEvent with body bigger than 1020 bytes are not re... + joyyoj + 2014-06-10 17:26:17 -0700 + Commit: 2966044, github.com/apache/spark/pull/951 + + [SQL] Add average overflow test case from #978 + egraldlo , Michael Armbrust + 2014-06-10 14:07:55 -0700 + Commit: 1abbde0, github.com/apache/spark/pull/1033 + + HOTFIX: Increase time limit for Bagel test + Ankur Dave + 2014-06-10 13:15:06 -0700 + Commit: 55a0e87, github.com/apache/spark/pull/1037 + + HOTFIX: Fix Python tests on Jenkins. + Patrick Wendell + 2014-06-10 13:13:17 -0700 + Commit: fb499be, github.com/apache/spark/pull/1036 + + [SPARK-2076][SQL] Pushdown the join filter & predication for outer join + Cheng Hao + 2014-06-10 12:59:52 -0700 + Commit: db0c038, github.com/apache/spark/pull/1015 + + [SPARK-1978] In some cases, spark-yarn does not automatically restart the failed container + witgo + 2014-06-10 10:34:57 -0500 + Commit: 884ca71, github.com/apache/spark/pull/921 + + Moved hiveOperators.scala to the right package folder + Cheng Lian + 2014-06-10 01:14:44 -0700 + Commit: a9a461c, github.com/apache/spark/pull/1029 + + [SPARK-1508][SQL] Add SQLConf to SQLContext. + Zongheng Yang + 2014-06-10 00:49:09 -0700 + Commit: 08ed9ad, github.com/apache/spark/pull/956 + + SPARK-1416: PySpark support for SequenceFile and Hadoop InputFormats + Nick Pentreath + 2014-06-09 22:21:03 -0700 + Commit: f971d6c, github.com/apache/spark/pull/455 + + Make sure that empty string is filtered out when we get the secondary jars from conf + DB Tsai + 2014-06-09 22:18:50 -0700 + Commit: 6f2db8c, github.com/apache/spark/pull/1027 + + [SPARK-1704][SQL] Fully support EXPLAIN commands as SchemaRDD. + Zongheng Yang + 2014-06-09 16:47:44 -0700 + Commit: a9ec033, github.com/apache/spark/pull/1003 + + [SQL] Simple framework for debugging query execution + Michael Armbrust + 2014-06-09 14:24:19 -0700 + Commit: c6e041d, github.com/apache/spark/pull/1005 + + [SPARK-1522] : YARN ClientBase throws a NPE if there is no YARN Application CP + Bernardo Gomez Palacio + 2014-06-09 16:14:54 -0500 + Commit: e273447, github.com/apache/spark/pull/433 + + Added a TaskSetManager unit test. + Kay Ousterhout + 2014-06-09 13:13:53 -0700 + Commit: 6cf335d, github.com/apache/spark/pull/1024 + + [SPARK-1495][SQL]add support for left semi join + Daoyuan , Michael Armbrust , Daoyuan Wang + 2014-06-09 11:31:36 -0700 + Commit: 0cf6002, github.com/apache/spark/pull/837 + + SPARK-1944 Document --verbose in spark-shell -h + Andrew Ash + 2014-06-09 10:21:21 -0700 + Commit: 35630c8, github.com/apache/spark/pull/1020 + + [SPARK-1308] Add getNumPartitions to pyspark RDD + Syed Hashmi + 2014-06-09 00:08:40 -0700 + Commit: 6113ac1, github.com/apache/spark/pull/995 + + Grammar: read -> reads + Andrew Ash + 2014-06-08 23:20:10 -0700 + Commit: 32ee9f0, github.com/apache/spark/pull/1016 + + [SPARK-2067] use relative path for Spark logo in UI + Neville Li + 2014-06-08 23:18:27 -0700 + Commit: 15ddbef, github.com/apache/spark/pull/1006 + + SPARK-1628 follow up: Improve RangePartitioner's documentation. + Reynold Xin + 2014-06-08 18:39:57 -0700 + Commit: 219dc00, github.com/apache/spark/pull/1012 + + Update run-example + maji2014 + 2014-06-08 15:14:27 -0700 + Commit: e9261d0, github.com/apache/spark/pull/1011 + + SPARK-1628: Add missing hashCode methods in Partitioner subclasses + zsxwing + 2014-06-08 14:18:52 -0700 + Commit: a71c6d1, github.com/apache/spark/pull/549 + + SPARK-1898: In deploy.yarn.Client, use YarnClient not YarnClientImpl + Colin Patrick McCabe + 2014-06-08 12:27:34 -0700 + Commit: ee96e94, github.com/apache/spark/pull/850 + + SPARK-2026: Maven Hadoop Profiles Should Set The Hadoop Version + Bernardo Gomez Palacio + 2014-06-08 01:24:52 -0700 + Commit: a338834, github.com/apache/spark/pull/998 + + SPARK-2056 Set RDD name to input path + Neville Li + 2014-06-07 16:22:26 -0700 + Commit: 7b877b2, github.com/apache/spark/pull/992 + + HOTFIX: Support empty body in merge script + Patrick Wendell + 2014-06-07 16:16:37 -0700 + Commit: 3ace10d, github.com/apache/spark/pull/1007 + + [SPARK-1994][SQL] Weird data corruption bug when running Spark SQL on data in HDFS + Michael Armbrust + 2014-06-07 14:20:33 -0700 + Commit: a6c72ab, github.com/apache/spark/pull/1004 + + [SPARK-1841]: update scalatest to version 2.1.5 + witgo + 2014-06-06 11:45:21 -0700 + Commit: 41c4a33, github.com/apache/spark/pull/713 + + [SPARK-2050 - 2][SQL] DIV and BETWEEN should not be case sensitive. + Michael Armbrust + 2014-06-06 11:31:37 -0700 + Commit: 8d21056, github.com/apache/spark/pull/994 + + [SPARK-1552] Fix type comparison bug in {map,outerJoin}Vertices + Ankur Dave + 2014-06-05 23:33:12 -0700 + Commit: 8d85359, github.com/apache/spark/pull/967 + + [SPARK-2050][SQL] LIKE, RLIKE and IN in HQL should not be case sensitive. + Michael Armbrust + 2014-06-05 23:20:59 -0700 + Commit: 41db44c, github.com/apache/spark/pull/989 + + SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys + Matei Zaharia + 2014-06-05 23:01:48 -0700 + Commit: b45c13e, github.com/apache/spark/pull/986 + + [SPARK-2025] Unpersist edges of previous graph in Pregel + Ankur Dave + 2014-06-05 17:45:38 -0700 + Commit: 9bad0b7, github.com/apache/spark/pull/972 + + Use pluggable clock in DAGSheduler #SPARK-2031 + CrazyJvm + 2014-06-05 17:44:46 -0700 + Commit: 3d3f8c8, github.com/apache/spark/pull/976 + + [SPARK-2041][SQL] Correctly analyze queries where columnName == tableName. + Michael Armbrust + 2014-06-05 17:42:08 -0700 + Commit: c7a183b, github.com/apache/spark/pull/985 + + Remove compile-scoped junit dependency. + Marcelo Vanzin + 2014-06-05 13:13:33 -0700 + Commit: 668cb1d, github.com/apache/spark/pull/794 + + sbt 0.13.X should be using sbt-assembly 0.11.X + Kalpit Shah + 2014-06-05 13:07:26 -0700 + Commit: 5473aa7, github.com/apache/spark/pull/555 + + HOTFIX: Remove generated-mima-excludes file after runing MIMA. + Patrick Wendell + 2014-06-05 13:06:46 -0700 + Commit: f6143f1, github.com/apache/spark/pull/971 + + [SPARK-2036] [SQL] CaseConversionExpression should check if the evaluated value is null. + Takuya UESHIN + 2014-06-05 12:00:31 -0700 + Commit: e4c11ee, github.com/apache/spark/pull/982 + + SPARK-1677: allow user to disable output dir existence checking + CodingCat + 2014-06-05 11:39:35 -0700 + Commit: 89cdbb0, github.com/apache/spark/pull/947 + + [SPARK-2029] Bump pom.xml version number of master branch to 1.1.0-SNAPSHOT. + Takuya UESHIN + 2014-06-05 11:27:33 -0700 + Commit: 7c16029, github.com/apache/spark/pull/974 + + Fix issue in ReplSuite with hadoop-provided profile. + Marcelo Vanzin + 2014-06-04 22:56:49 -0700 + Commit: b77c19b, github.com/apache/spark/pull/781 + + Minor: Fix documentation error from apache/spark#946 + Ankur Dave + 2014-06-04 16:45:53 -0700 + Commit: abea2d4, github.com/apache/spark/pull/970 + + SPARK-1790: Update EC2 scripts to support r3 instance types + Varakhedi Sujeet + 2014-06-04 16:01:56 -0700 + Commit: 11ded3f, github.com/apache/spark/pull/960 + + SPARK-1518: FileLogger: Fix compile against Hadoop trunk + Colin McCabe + 2014-06-04 15:56:29 -0700 + Commit: 1765c8d, github.com/apache/spark/pull/898 + + [SPARK-1752][MLLIB] Standardize text format for vectors and labeled points + Xiangrui Meng + 2014-06-04 12:56:56 -0700 + Commit: 189df16, github.com/apache/spark/pull/685 + + SPARK-1973. Add randomSplit to JavaRDD (with tests, and tidy Java tests) + Sean Owen , Xiangrui Meng + 2014-06-04 11:27:08 -0700 + Commit: d341b17, github.com/apache/spark/pull/919 + + [MLLIB] set RDD names in ALS + Neville Li + 2014-06-04 01:51:34 -0700 + Commit: b8d2580, github.com/apache/spark/pull/966 + + [SPARK-1817] RDD.zip() should verify partition sizes for each partition + Kan Zhang + 2014-06-03 22:47:18 -0700 + Commit: c402a4a, github.com/apache/spark/pull/944 + + SPARK-1806 (addendum) Use non-deprecated methods in Mesos 0.18 + Sean Owen + 2014-06-03 22:37:20 -0700 + Commit: 4ca0625, github.com/apache/spark/pull/920 + + Update spark-ec2 scripts for 1.0.0 on master + Aaron Davidson + 2014-06-03 22:33:04 -0700 + Commit: ab7c62d, github.com/apache/spark/pull/938 + + Enable repartitioning of graph over different number of partitions + Joseph E. Gonzalez + 2014-06-03 20:49:14 -0700 + Commit: 5284ca7, github.com/apache/spark/pull/719 + + use env default python in merge_spark_pr.py + Xiangrui Meng + 2014-06-03 18:53:13 -0700 + Commit: e8d93ee, github.com/apache/spark/pull/965 + + SPARK-1941: Update streamlib to 2.7.0 and use HyperLogLogPlus instead of HyperLogLog. + Reynold Xin + 2014-06-03 18:37:40 -0700 + Commit: 1faef14, github.com/apache/spark/pull/897 + + [SPARK-1161] Add saveAsPickleFile and SparkContext.pickleFile in Python + Kan Zhang + 2014-06-03 18:18:25 -0700 + Commit: 21e40ed, github.com/apache/spark/pull/755 + + Fixed a typo + DB Tsai + 2014-06-03 18:10:58 -0700 + Commit: f4dd665, github.com/apache/spark/pull/959 + + [SPARK-1991] Support custom storage levels for vertices and edges + Ankur Dave + 2014-06-03 14:54:26 -0700 + Commit: b1feb60, github.com/apache/spark/pull/946 + + Synthetic GraphX Benchmark + Joseph E. Gonzalez , Ankur Dave + 2014-06-03 14:14:48 -0700 + Commit: 894ecde, github.com/apache/spark/pull/720 + + fix java.lang.ClassCastException + baishuo(白硕) + 2014-06-03 13:39:47 -0700 + Commit: aa41a52, github.com/apache/spark/pull/949 + + [SPARK-1468] Modify the partition function used by partitionBy. + Erik Selin + 2014-06-03 13:31:16 -0700 + Commit: 8edc9d0, github.com/apache/spark/pull/371 + + Add support for Pivotal HD in the Maven build: SPARK-1992 + tzolov + 2014-06-03 13:26:29 -0700 + Commit: b1f2853, github.com/apache/spark/pull/942 + + [SPARK-1912] fix compress memory issue during reduce + Wenchen Fan(Cloud) + 2014-06-03 13:18:20 -0700 + Commit: 45e9bc8, github.com/apache/spark/pull/860 + + SPARK-2001 : Remove docs/spark-debugger.md from master + Henry Saputra + 2014-06-03 13:03:51 -0700 + Commit: 6c044ed, github.com/apache/spark/pull/953 + + [SPARK-1942] Stop clearing spark.driver.port in unit tests + Syed Hashmi , CodingCat + 2014-06-03 12:04:47 -0700 + Commit: 7782a30, github.com/apache/spark/pull/943 + + Avoid dynamic dispatching when unwrapping Hive data. + Cheng Lian + 2014-06-02 19:20:23 -0700 + Commit: 862283e, github.com/apache/spark/pull/935 + + [SPARK-1995][SQL] system function upper and lower can be supported + egraldlo + 2014-06-02 18:02:57 -0700 + Commit: ec8be27, github.com/apache/spark/pull/936 + + [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan. + Cheng Lian + 2014-06-02 12:09:43 -0700 + Commit: d000ca9, github.com/apache/spark/pull/939 + + [SPARK-1553] Alternating nonnegative least-squares + Tor Myklebust + 2014-06-02 11:48:09 -0700 + Commit: 9a5d482, github.com/apache/spark/pull/460 + + Add landmark-based Shortest Path algorithm to graphx.lib + Ankur Dave , Andres Perez + 2014-06-02 00:00:24 -0700 + Commit: 9535f40, github.com/apache/spark/pull/933 + + Better explanation for how to use MIMA excludes. + Patrick Wendell + 2014-06-01 17:27:05 -0700 + Commit: d17d221, github.com/apache/spark/pull/937 + + Made spark_ec2.py PEP8 compliant. + Reynold Xin + 2014-06-01 15:39:04 -0700 + Commit: eea3aab, github.com/apache/spark/pull/891 + + updated java code blocks in spark SQL guide such that ctx will refer to ... + Yadid Ayzenberg + 2014-05-31 19:44:13 -0700 + Commit: 366c0c4, github.com/apache/spark/pull/932 + + SPARK-1917: fix PySpark import of scipy.special functions + Uri Laserson + 2014-05-31 14:59:09 -0700 + Commit: 5e98967, github.com/apache/spark/pull/866 + + Improve maven plugin configuration + witgo + 2014-05-31 14:36:27 -0700 + Commit: d8c005d, github.com/apache/spark/pull/786 + + SPARK-1839: PySpark RDD#take() shouldn't always read from driver + Aaron Davidson + 2014-05-31 13:04:57 -0700 + Commit: 9909efc, github.com/apache/spark/pull/922 + + Super minor: Close inputStream in SparkSubmitArguments + Aaron Davidson + 2014-05-31 12:36:58 -0700 + Commit: 7d52777, github.com/apache/spark/pull/914 + + [SQL] SPARK-1964 Add timestamp to hive metastore type parser. + Michael Armbrust + 2014-05-31 12:34:22 -0700 + Commit: 1a0da0e, github.com/apache/spark/pull/913 + + Optionally include Hive as a dependency of the REPL. + Michael Armbrust + 2014-05-31 12:24:35 -0700 + Commit: 7463cd2, github.com/apache/spark/pull/801 + + [SPARK-1947] [SQL] Child of SumDistinct or Average should be widened to prevent overflows the same as Sum. + Takuya UESHIN + 2014-05-31 11:30:03 -0700 + Commit: 3ce8149, github.com/apache/spark/pull/902 + + correct tiny comment error + Chen Chao + 2014-05-31 00:06:49 -0700 + Commit: 9ecc40d, github.com/apache/spark/pull/928 + + [SPARK-1959] String "NULL" shouldn't be interpreted as null value + Cheng Lian + 2014-05-30 22:13:11 -0700 + Commit: cf98960, github.com/apache/spark/pull/909 + + SPARK-1976: fix the misleading part in streaming docs + CodingCat + 2014-05-30 22:06:08 -0700 + Commit: 41bfdda, github.com/apache/spark/pull/924 + + updated link to mailing list + nchammas + 2014-05-30 22:04:57 -0700 + Commit: 23ae366, github.com/apache/spark/pull/923 + + Typo: and -> an + Andrew Ash + 2014-05-30 22:02:04 -0700 + Commit: 9c1f204, github.com/apache/spark/pull/927 + + [SPARK-1901] worker should make sure executor has exited before updating executor's info + Zhen Peng + 2014-05-30 10:11:02 -0700 + Commit: ff562b2, github.com/apache/spark/pull/854 + + [SPARK-1971] Update MIMA to compare against Spark 1.0.0 + Prashant Sharma + 2014-05-30 01:13:51 -0700 + Commit: 79fa8fd, github.com/apache/spark/pull/910 + + [SPARK-1566] consolidate programming guide, and general doc updates + Matei Zaharia + 2014-05-30 00:34:33 -0700 + Commit: c8bf413, github.com/apache/spark/pull/896 + + [SPARK-1820] Make GenerateMimaIgnore @DeveloperApi annotation aware. + Prashant Sharma , nikhil7sh + 2014-05-29 23:20:20 -0700 + Commit: eeee978, github.com/apache/spark/pull/904 + + initial version of LPA + Ankur Dave , haroldsultan , Harold Sultan + 2014-05-29 15:39:25 -0700 + Commit: b7e28fa, github.com/apache/spark/pull/905 + + [SPARK-1368][SQL] Optimized HiveTableScan + Cheng Lian + 2014-05-29 15:24:03 -0700 + Commit: 8f7141f, github.com/apache/spark/pull/758 + + SPARK-1935: Explicitly add commons-codec 1.5 as a dependency. + Yin Huai + 2014-05-29 09:07:39 -0700 + Commit: 60b89fe, github.com/apache/spark/pull/889 + + Added doctest and method description in context.py + Jyotiska NK + 2014-05-28 23:08:39 -0700 + Commit: 9cff1dd, github.com/apache/spark/pull/187 + + [SPARK-1712]: TaskDescription instance is too big causes Spark to hang + witgo + 2014-05-28 15:57:05 -0700 + Commit: 4dbb27b, github.com/apache/spark/pull/694 + + Spark 1916 + David Lemieux + 2014-05-28 15:50:35 -0700 + Commit: 4312cf0, github.com/apache/spark/pull/865 + + Organize configuration docs + Patrick Wendell + 2014-05-28 15:49:54 -0700 + Commit: 7801d44, github.com/apache/spark/pull/880 + + Fix doc about NetworkWordCount/JavaNetworkWordCount usage of spark streaming + jmu + 2014-05-27 22:41:47 -0700 + Commit: 82eadc3, github.com/apache/spark/pull/826 + + [SPARK-1938] [SQL] ApproxCountDistinctMergeFunction should return Int value. + Takuya UESHIN + 2014-05-27 22:17:50 -0700 + Commit: 9df8683, github.com/apache/spark/pull/893 + + [SQL] SPARK-1922 + LY Lai + 2014-05-27 16:08:38 -0700 + Commit: 0682567, github.com/apache/spark/pull/873 + + [SPARK-1915] [SQL] AverageFunction should not count if the evaluated value is null. + Takuya UESHIN + 2014-05-27 14:55:23 -0700 + Commit: 3b0baba, github.com/apache/spark/pull/862 + + [SPARK-1926] [SQL] Nullability of Max/Min/First should be true. + Takuya UESHIN + 2014-05-27 14:53:57 -0700 + Commit: d1375a2, github.com/apache/spark/pull/881 + + bugfix worker DriverStateChanged state should match DriverState.FAILED + lianhuiwang + 2014-05-27 11:53:38 -0700 + Commit: 95e4c9c, github.com/apache/spark/pull/864 + + SPARK-1932: Fix race conditions in onReceiveCallback and cachedPeers + zsxwing + 2014-05-26 23:17:39 -0700 + Commit: 549830b, github.com/apache/spark/pull/887 + + SPARK-1933: Throw a more meaningful exception when a directory is passed to addJar/addFile. + Reynold Xin + 2014-05-26 22:05:23 -0700 + Commit: 90e281b, github.com/apache/spark/pull/888 + + Updated dev Python scripts to make them PEP8 compliant. + Reynold Xin + 2014-05-26 21:40:52 -0700 + Commit: 9ed3719, github.com/apache/spark/pull/875 + + Fixed the error message for OutOfMemoryError in DAGScheduler. + Reynold Xin + 2014-05-26 21:31:27 -0700 + Commit: ef690e1 + + SPARK-1929 DAGScheduler suspended by local task OOM + Zhen Peng + 2014-05-26 21:30:25 -0700 + Commit: 8d271c9, github.com/apache/spark/pull/883 + + [SPARK-1931] Reconstruct routing tables in Graph.partitionBy + Ankur Dave + 2014-05-26 16:10:22 -0700 + Commit: 56c771c, github.com/apache/spark/pull/885 + + SPARK-1925: Replace '&' with '&&' + zsxwing + 2014-05-26 14:34:58 -0700 + Commit: cb7fe50, github.com/apache/spark/pull/879 + + Fix scalastyle warnings in yarn alpha + witgo + 2014-05-26 13:16:35 -0700 + Commit: bee6c4f, github.com/apache/spark/pull/884 + + [SPARK-1914] [SQL] Simplify CountFunction not to traverse to evaluate all child expressions. + Takuya UESHIN + 2014-05-26 00:17:20 -0700 + Commit: d6395d8, github.com/apache/spark/pull/861 + + HOTFIX: Add no-arg SparkContext constructor in Java + Patrick Wendell + 2014-05-25 20:13:32 -0700 + Commit: b6d22af, github.com/apache/spark/pull/878 + + [SQL] Minor: Introduce SchemaRDD#aggregate() for simple aggregations + Aaron Davidson + 2014-05-25 18:37:44 -0700 + Commit: c3576ff, github.com/apache/spark/pull/874 + + SPARK-1903 Document Spark's network connections + Andrew Ash + 2014-05-25 17:15:47 -0700 + Commit: 0659529, github.com/apache/spark/pull/856 + + Fix PEP8 violations in Python mllib. + Reynold Xin + 2014-05-25 17:15:01 -0700 + Commit: d33d3c6, github.com/apache/spark/pull/871 + + Python docstring update for sql.py. + Reynold Xin + 2014-05-25 16:04:17 -0700 + Commit: 14f0358, github.com/apache/spark/pull/869 + + Fix PEP8 violations in examples/src/main/python. + Reynold Xin + 2014-05-25 14:48:27 -0700 + Commit: d79c2b2, github.com/apache/spark/pull/870 + + Added license header for tox.ini. + Reynold Xin + 2014-05-25 01:47:08 -0700 + Commit: 55fddf9 + + SPARK-1822: Some minor cleanup work on SchemaRDD.count() + Reynold Xin + 2014-05-25 01:44:49 -0700 + Commit: d66642e, github.com/apache/spark/pull/868 + + Added PEP8 style configuration file. + Reynold Xin + 2014-05-25 01:32:15 -0700 + Commit: 5c7faec, github.com/apache/spark/pull/872 + + [SPARK-1822] SchemaRDD.count() should use query optimizer + Kan Zhang + 2014-05-25 00:06:42 -0700 + Commit: 6052db9, github.com/apache/spark/pull/841 + + spark-submit: add exec at the end of the script + Colin Patrick Mccabe + 2014-05-24 22:39:27 -0700 + Commit: 6e9fb63, github.com/apache/spark/pull/858 + + [SPARK-1913][SQL] Bug fix: column pruning error in Parquet support + Cheng Lian + 2014-05-24 20:42:01 -0700 + Commit: 5afe6af, github.com/apache/spark/pull/863 + + [SPARK-1886] check executor id existence when executor exit + Zhen Peng + 2014-05-24 20:40:19 -0700 + Commit: 4e4831b, github.com/apache/spark/pull/827 + + SPARK-1911: Emphasize that Spark jars should be built with Java 6. + Patrick Wendell + 2014-05-24 18:27:00 -0700 + Commit: 75a0327, github.com/apache/spark/pull/859 + + [SPARK-1900 / 1918] PySpark on YARN is broken + Andrew Or + 2014-05-24 18:01:49 -0700 + Commit: 5081a0a, github.com/apache/spark/pull/853 + + Update LBFGSSuite.scala + baishuo(白硕) + 2014-05-23 13:02:40 -0700 + Commit: a08262d, github.com/apache/spark/pull/815 + + Updated scripts for auditing releases + Tathagata Das + 2014-05-22 20:48:55 -0700 + Commit: b2bdd0e, github.com/apache/spark/pull/844 + + [SPARK-1896] Respect spark.master (and --master) before MASTER in spark-shell + Andrew Or + 2014-05-22 20:32:27 -0700 + Commit: cce7745, github.com/apache/spark/pull/846 + + [SPARK-1897] Respect spark.jars (and --jars) in spark-shell + Andrew Or + 2014-05-22 20:25:41 -0700 + Commit: 8edbee7, github.com/apache/spark/pull/849 + + Fix UISuite unit test that fails under Jenkins contention + Aaron Davidson + 2014-05-22 15:11:05 -0700 + Commit: f9f5fd5, github.com/apache/spark/pull/857 + + [SPARK-1870] Make spark-submit --jars work in yarn-cluster mode. + Xiangrui Meng + 2014-05-22 01:52:50 -0700 + Commit: dba3140, github.com/apache/spark/pull/848 + + Configuration documentation updates + Reynold Xin + 2014-05-21 18:49:12 -0700 + Commit: 2a948e7, github.com/apache/spark/pull/851 + + [SPARK-1889] [SQL] Apply splitConjunctivePredicates to join condition while finding join ke... + Takuya UESHIN + 2014-05-21 15:37:47 -0700 + Commit: bb88875, github.com/apache/spark/pull/836 + + [SPARK-1519] Support minPartitions param of wholeTextFiles() in PySpark + Kan Zhang + 2014-05-21 13:26:53 -0700 + Commit: f18fd05, github.com/apache/spark/pull/697 + + [Typo] Stoped -> Stopped + Andrew Or + 2014-05-21 11:59:05 -0700 + Commit: ba5d4a9, github.com/apache/spark/pull/847 + + [Minor] Move JdbcRDDSuite to the correct package + Andrew Or + 2014-05-21 01:25:10 -0700 + Commit: 7c79ef7, github.com/apache/spark/pull/839 + + [Docs] Correct example of creating a new SparkConf + Andrew Or + 2014-05-21 01:23:34 -0700 + Commit: 1014668, github.com/apache/spark/pull/842 + + [SPARK-1250] Fixed misleading comments in bin/pyspark, bin/spark-class + Sumedh Mungee + 2014-05-21 01:22:25 -0700 + Commit: 6e33738, github.com/apache/spark/pull/843 + + [Hotfix] Blacklisted flaky HiveCompatibility test + Tathagata Das + 2014-05-20 10:27:12 -0700 + Commit: 7f0cfe4, github.com/apache/spark/pull/838 + + [Spark 1877] ClassNotFoundException when loading RDD with serialized objects + Tathagata Das , Ghidireac + 2014-05-19 22:36:24 -0700 + Commit: 52eb54d, github.com/apache/spark/pull/835 + + [SPARK-1874][MLLIB] Clean up MLlib sample data + Xiangrui Meng + 2014-05-19 21:29:33 -0700 + Commit: bcb9dce, github.com/apache/spark/pull/833 + + SPARK-1689: Spark application should die when removed by Master + Aaron Davidson + 2014-05-19 20:55:26 -0700 + Commit: b0ce22e, github.com/apache/spark/pull/832 + + [SPARK-1875]NoClassDefFoundError: StringUtils when building with hadoop 1.x and hive + witgo + 2014-05-19 19:40:29 -0700 + Commit: 6a2c5c6, github.com/apache/spark/pull/824 + + SPARK-1879. Increase MaxPermSize since some of our builds have many classes + Matei Zaharia + 2014-05-19 18:42:28 -0700 + Commit: 5af99d7, github.com/apache/spark/pull/823 + + SPARK-1878: Fix the incorrect initialization order + zsxwing + 2014-05-19 16:41:31 -0700 + Commit: 1811ba8, github.com/apache/spark/pull/822 + + [SPARK-1876] Windows fixes to deal with latest distribution layout changes + Matei Zaharia + 2014-05-19 15:02:35 -0700 + Commit: 7b70a70, github.com/apache/spark/pull/819 + + [WIP][SPARK-1871][MLLIB] Improve MLlib guide for v1.0 + Xiangrui Meng + 2014-05-18 17:00:57 -0700 + Commit: df0aa83, github.com/apache/spark/pull/816 + + SPARK-1873: Add README.md file when making distributions + Patrick Wendell + 2014-05-18 16:51:53 -0700 + Commit: 4ce4793, github.com/apache/spark/pull/818 + + Fix spark-submit path in spark-shell & pyspark + Neville Li + 2014-05-18 13:31:23 -0700 + Commit: ebcd2d6, github.com/apache/spark/pull/812 + + Make deprecation warning less severe + Patrick Wendell + 2014-05-16 22:58:47 -0700 + Commit: 442808a, github.com/apache/spark/pull/810 + + [SPARK-1824] Remove from Python examples + Andrew Or + 2014-05-16 22:36:23 -0700 + Commit: cf6cbe9, github.com/apache/spark/pull/802 + + [SPARK-1808] Route bin/pyspark through Spark submit + Andrew Or + 2014-05-16 22:34:38 -0700 + Commit: 4b8ec6f, github.com/apache/spark/pull/799 + + Version bump of spark-ec2 scripts + Patrick Wendell + 2014-05-16 21:42:14 -0700 + Commit: c0ab85d, github.com/apache/spark/pull/809 + + SPARK-1864 Look in spark conf instead of system properties when propagating configuration to executors. + Michael Armbrust + 2014-05-16 20:25:10 -0700 + Commit: a80a6a1, github.com/apache/spark/pull/808 + + Tweaks to Mesos docs + Matei Zaharia + 2014-05-16 17:35:05 -0700 + Commit: fed6303, github.com/apache/spark/pull/806 + + SPARK-1487 [SQL] Support record filtering via predicate pushdown in Parquet + Andre Schumacher + 2014-05-16 13:41:41 -0700 + Commit: 40d6acd, github.com/apache/spark/pull/511 + + [SQL] Implement between in hql + Michael Armbrust + 2014-05-16 11:47:00 -0700 + Commit: 032d663, github.com/apache/spark/pull/804 + + bugfix: overflow of graphx Edge compare function + Zhen Peng + 2014-05-16 11:37:18 -0700 + Commit: fa6de40, github.com/apache/spark/pull/769 + + HOTFIX: Duplication of hbase version + Patrick Wendell + 2014-05-15 23:33:27 -0700 + Commit: e304eb9 + + SPARK-1862: Support for MapR in the Maven build. + Patrick Wendell + 2014-05-15 23:31:43 -0700 + Commit: 17702e2, github.com/apache/spark/pull/803 + + [Spark-1461] Deferred Expression Evaluation (short-circuit evaluation) + Cheng Hao + 2014-05-15 22:12:34 -0700 + Commit: a20fea9, github.com/apache/spark/pull/446 + + SPARK-1860: Do not cleanup application work/ directories by default + Aaron Davidson + 2014-05-15 21:37:58 -0700 + Commit: bb98eca, github.com/apache/spark/pull/800 + + Typos in Spark + Huajian Mao + 2014-05-15 18:20:16 -0700 + Commit: 94c5139, github.com/apache/spark/pull/798 + + Fixes a misplaced comment. + Prashant Sharma + 2014-05-15 16:58:37 -0700 + Commit: e1e3416, github.com/apache/spark/pull/788 + + [SQL] Fix tiny/small ints from HiveMetastore. + Michael Armbrust + 2014-05-15 16:50:42 -0700 + Commit: a4aafe5, github.com/apache/spark/pull/797 + + SPARK-1803 Replaced colon in filenames with a dash + Stevo Slavić , Stevo Slavic + 2014-05-15 16:44:14 -0700 + Commit: e66e31b, github.com/apache/spark/pull/739 + + SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro file... + Sandy Ryza + 2014-05-15 16:35:39 -0700 + Commit: 08e7606, github.com/apache/spark/pull/795 + + [SPARK-1741][MLLIB] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans + Xiangrui Meng + 2014-05-15 11:59:59 -0700 + Commit: d52761d, github.com/apache/spark/pull/670 + + [SPARK-1819] [SQL] Fix GetField.nullable. + Takuya UESHIN + 2014-05-15 11:21:33 -0700 + Commit: 94c9d6f, github.com/apache/spark/pull/757 + + [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ... + Takuya UESHIN + 2014-05-15 11:20:21 -0700 + Commit: db8cc6f, github.com/apache/spark/pull/790 + + SPARK-1846 Ignore logs directory in RAT checks + Andrew Ash + 2014-05-15 11:05:39 -0700 + Commit: 3abe2b7, github.com/apache/spark/pull/793 + + HOTFIX: Don't build Javadoc in Maven when creating releases. + Patrick Wendell + 2014-05-14 23:48:03 -0700 + Commit: 514157f + + fix different versions of commons-lang dependency and apache/spark#746 addendum + witgo + 2014-05-14 22:26:26 -0700 + Commit: bae07e3, github.com/apache/spark/pull/754 + + Package docs + Prashant Sharma , Patrick Wendell + 2014-05-14 22:24:41 -0700 + Commit: 4632427, github.com/apache/spark/pull/785 + + Documentation: Encourage use of reduceByKey instead of groupByKey. + Patrick Wendell + 2014-05-14 22:24:04 -0700 + Commit: 21570b4, github.com/apache/spark/pull/784 + + Add language tabs and Python version to interactive part of quick-start + Matei Zaharia + 2014-05-14 21:45:20 -0700 + Commit: f10de04, github.com/apache/spark/pull/782 + + [SPARK-1840] SparkListenerBus prints out scary error message when terminated normally + Tathagata Das + 2014-05-14 21:13:41 -0700 + Commit: ad4e60e, github.com/apache/spark/pull/783 + + default task number misleading in several places + Chen Chao + 2014-05-14 18:20:20 -0700 + Commit: 2f63995, github.com/apache/spark/pull/766 + + [SPARK-1826] fix the head notation of package object dsl + wangfei + 2014-05-14 17:59:11 -0700 + Commit: 44165fc, github.com/apache/spark/pull/765 + + [Typo] propertes -> properties + andrewor14 + 2014-05-14 17:54:53 -0700 + Commit: 9ad096d, github.com/apache/spark/pull/780 + + [SPARK-1696][MLLIB] use alpha in dense dspr + Xiangrui Meng + 2014-05-14 17:18:30 -0700 + Commit: e3d72a7, github.com/apache/spark/pull/778 + + String interpolation + some other small changes + Jacek Laskowski + 2014-05-14 15:45:52 -0700 + Commit: 601e371, github.com/apache/spark/pull/748 + + [FIX] do not load defaults when testing SparkConf in pyspark + Xiangrui Meng + 2014-05-14 14:57:17 -0700 + Commit: 94c6c06, github.com/apache/spark/pull/775 + + SPARK-1833 - Have an empty SparkContext constructor. + Patrick Wendell + 2014-05-14 12:53:30 -0700 + Commit: 65533c7, github.com/apache/spark/pull/774 + + SPARK-1829 Sub-second durations shouldn't round to "0 s" + Andrew Ash + 2014-05-14 12:01:14 -0700 + Commit: a3315d7, github.com/apache/spark/pull/768 + + Fix: sbt test throw an java.lang.OutOfMemoryError: PermGen space + witgo + 2014-05-14 11:19:26 -0700 + Commit: fde82c1, github.com/apache/spark/pull/773 + + [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler + Mark Hamstra + 2014-05-14 10:07:25 -0700 + Commit: 17f3075, github.com/apache/spark/pull/622 + + SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies + Patrick Wendell + 2014-05-14 09:51:01 -0700 + Commit: d58cb33, github.com/apache/spark/pull/767 + + SPARK-1818 Freshen Mesos documentation + Andrew Ash + 2014-05-14 09:45:33 -0700 + Commit: d1d41cc, github.com/apache/spark/pull/756 + + SPARK-1827. LICENSE and NOTICE files need a refresh to contain transitive dependency info + Sean Owen + 2014-05-14 09:38:33 -0700 + Commit: 2e5a7cd, github.com/apache/spark/pull/770 + + Fixed streaming examples docs to use run-example instead of spark-submit + Tathagata Das + 2014-05-14 04:17:32 -0700 + Commit: 68f28da, github.com/apache/spark/pull/722 + + [SPARK-1769] Executor loss causes NPE race condition + Andrew Or + 2014-05-14 00:54:33 -0700 + Commit: 69f7502, github.com/apache/spark/pull/762 + + Fix dep exclusion: avro-ipc, not avro, depends on netty. + Marcelo Vanzin + 2014-05-14 00:37:57 -0700 + Commit: 54ae832, github.com/apache/spark/pull/763 + + SPARK-1801. expose InterruptibleIterator and TaskKilledException in deve... + Koert Kuipers + 2014-05-14 00:10:12 -0700 + Commit: b22952f, github.com/apache/spark/pull/764 + + [SQL] Improve column pruning. + Michael Armbrust + 2014-05-13 23:27:22 -0700 + Commit: 6ce0884, github.com/apache/spark/pull/729 + + Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per partition" + Patrick Wendell + 2014-05-13 23:24:51 -0700 + Commit: 7bb9a52 + + Implement ApproximateCountDistinct for SparkSql + larvaboy + 2014-05-13 21:26:08 -0700 + Commit: c33b8dc, github.com/apache/spark/pull/737 + + [SPARK-1784] Add a new partitioner to allow specifying # of keys per partition + Syed Hashmi + 2014-05-13 21:24:23 -0700 + Commit: 92cebad, github.com/apache/spark/pull/721 + + [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. + Michael Armbrust + 2014-05-13 21:23:51 -0700 + Commit: 4423386, github.com/apache/spark/pull/761 + + [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePath + Ye Xianjin + 2014-05-13 19:03:51 -0700 + Commit: 753b04d, github.com/apache/spark/pull/436 + + [SPARK-1816] LiveListenerBus dies if a listener throws an exception + Andrew Or + 2014-05-13 18:32:32 -0700 + Commit: 5c0dafc, github.com/apache/spark/pull/759 + + SPARK-1791 - SVM implementation does not use threshold parameter + Andrew Tulloch + 2014-05-13 17:31:27 -0700 + Commit: d1e4874, github.com/apache/spark/pull/725 + + SPARK-571: forbid return statements in cleaned closures + William Benton + 2014-05-13 13:45:23 -0700 + Commit: 16ffadc, github.com/apache/spark/pull/717 + + BUILD: Add more content to make-distribution.sh. + Patrick Wendell + 2014-05-12 23:02:54 -0700 + Commit: 52d9052 + + SPARK-1815. SparkContext should not be marked DeveloperApi + Sandy Ryza + 2014-05-12 20:08:30 -0700 + Commit: 2792bd0, github.com/apache/spark/pull/753 + + [SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. + Andrew Or + 2014-05-12 19:44:14 -0700 + Commit: 2ffd1ea, github.com/apache/spark/pull/701 + + [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking around + Andrew Or + 2014-05-12 19:42:35 -0700 + Commit: ba96bb3, github.com/apache/spark/pull/751 + + SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile() + Andrew Ash , Michael Armbrust + 2014-05-12 19:23:39 -0700 + Commit: 156df87, github.com/apache/spark/pull/690 + + Modify a typo in monitoring.md + Kousuke Saruta + 2014-05-12 19:21:06 -0700 + Commit: 9cf9f18, github.com/apache/spark/pull/698 + + L-BFGS Documentation + DB Tsai + 2014-05-12 19:20:24 -0700 + Commit: 5c2275d, github.com/apache/spark/pull/702 + + Typo: resond -> respond + Andrew Ash + 2014-05-12 18:46:28 -0700 + Commit: a5150d1, github.com/apache/spark/pull/743 + + [SQL] Make Hive Metastore conversion functions publicly visible. + Michael Armbrust + 2014-05-12 18:40:30 -0700 + Commit: 2f1a337, github.com/apache/spark/pull/750 + + Adding hadoop-2.2 profile to the build + Patrick Wendell + 2014-05-12 15:40:48 -0700 + Commit: 3e13b8c + + [SPARK-1736] Spark submit for Windows + Andrew Or + 2014-05-12 17:39:40 -0700 + Commit: beb9cba, github.com/apache/spark/pull/745 + + SPARK-1802. (Addendium) Audit dependency graph when Spark is built with -Pyarn + Sean Owen + 2014-05-12 17:35:29 -0700 + Commit: 4b31f4e, github.com/apache/spark/pull/746 + + SPARK-1623: Use File objects instead of String's in HTTPBroadcast + Patrick Wendell + 2014-05-12 17:27:28 -0700 + Commit: 925d8b2, github.com/apache/spark/pull/749 + + Rename testExecutorEnvs --> executorEnvs. + Patrick Wendell + 2014-05-12 17:09:13 -0700 + Commit: 3ce526b, github.com/apache/spark/pull/747 + + SPARK-1802. Audit dependency graph when Spark is built with -Phive + Sean Owen + 2014-05-12 14:17:25 -0700 + Commit: 8586bf5, github.com/apache/spark/pull/744 + + SPARK-1798. Tests should clean up temp files + Sean Owen + 2014-05-12 14:16:19 -0700 + Commit: 7120a29, github.com/apache/spark/pull/732 + + BUILD: Include Hive with default packages when creating a release + Patrick Wendell + 2014-05-12 13:20:23 -0700 + Commit: 1e4a65e + + SPARK-1786: Reopening PR 724 + Ankur Dave , Joseph E. Gonzalez + 2014-05-12 13:05:24 -0700 + Commit: 0e2bde2, github.com/apache/spark/pull/742 + + SPARK-1806: Upgrade Mesos dependency to 0.18.1 + Bernardo Gomez Palacio + 2014-05-12 11:10:28 -0700 + Commit: d9c97ba, github.com/apache/spark/pull/741 + + SPARK-1772 Stop catching Throwable, let Executors die + Aaron Davidson + 2014-05-12 11:08:52 -0700 + Commit: 3af1f38, github.com/apache/spark/pull/715 + + Revert "SPARK-1786: Edge Partition Serialization" + Patrick Wendell + 2014-05-12 10:49:03 -0700 + Commit: af15c82 + + SPARK-1786: Edge Partition Serialization + Ankur Dave , Joseph E. Gonzalez + 2014-05-11 19:20:42 -0700 + Commit: a6b02fb, github.com/apache/spark/pull/724 + + Fix error in 2d Graph Partitioner + Joseph E. Gonzalez + 2014-05-11 18:33:46 -0700 + Commit: f938a15, github.com/apache/spark/pull/709 + + SPARK-1652: Set driver memory correctly in spark-submit. + Patrick Wendell + 2014-05-11 18:17:34 -0700 + Commit: 05c9aa9, github.com/apache/spark/pull/730 + + SPARK-1770: Load balance elements when repartitioning. + Patrick Wendell + 2014-05-11 17:11:55 -0700 + Commit: 7d9cc92, github.com/apache/spark/pull/727 + + remove outdated runtime Information scala home + witgo + 2014-05-11 14:34:27 -0700 + Commit: 6bee01d, github.com/apache/spark/pull/728 + + Enabled incremental build that comes with sbt 0.13.2 + Prashant Sharma + 2014-05-10 21:08:04 -0700 + Commit: 70bcdef, github.com/apache/spark/pull/525 + + [SPARK-1774] Respect SparkSubmit --jars on YARN (client) + Andrew Or + 2014-05-10 20:58:02 -0700 + Commit: 83e0424, github.com/apache/spark/pull/710 + + SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite failure + Sean Owen + 2014-05-10 20:50:40 -0700 + Commit: 2b7bd29, github.com/apache/spark/pull/723 + + Unify GraphImpl RDDs + other graph load optimizations + Ankur Dave + 2014-05-10 14:48:07 -0700 + Commit: 905173d, github.com/apache/spark/pull/497 + + [SPARK-1690] Tolerating empty elements when saving Python RDD to text files + Kan Zhang + 2014-05-10 14:01:08 -0700 + Commit: 6c2691d, github.com/apache/spark/pull/644 + + Add Python includes to path before depickling broadcast values + Bouke van der Bijl + 2014-05-10 13:02:13 -0700 + Commit: 3776f2f, github.com/apache/spark/pull/656 + + fix broken in link in python docs + Andy Konwinski + 2014-05-10 12:46:51 -0700 + Commit: c05d11b, github.com/apache/spark/pull/650 + + SPARK-1708. Add a ClassTag on Serializer and things that depend on it + Matei Zaharia + 2014-05-10 12:10:24 -0700 + Commit: 7eefc9d, github.com/apache/spark/pull/700 + + [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD. + Takuya UESHIN + 2014-05-10 12:03:27 -0700 + Commit: 8e94d27, github.com/apache/spark/pull/711 + + [SQL] Upgrade parquet library. + Michael Armbrust + 2014-05-10 11:48:01 -0700 + Commit: 4d60553, github.com/apache/spark/pull/684 + + [SPARK-1644] The org.datanucleus:* should not be packaged into spark-assembly-*.jar + witgo + 2014-05-10 10:15:04 -0700 + Commit: 5615108, github.com/apache/spark/pull/688 + + SPARK-1686: keep schedule() calling in the main thread + CodingCat + 2014-05-09 21:50:23 -0700 + Commit: 2f452cb, github.com/apache/spark/pull/639 + + SPARK-1770: Revert accidental(?) fix + Aaron Davidson + 2014-05-09 14:51:34 -0700 + Commit: 59577df, github.com/apache/spark/pull/716 + + [SPARK-1760]: fix building spark with maven documentation + witgo + 2014-05-09 01:51:26 -0700 + Commit: bd67551, github.com/apache/spark/pull/712 + + Converted bang to ask to avoid scary warning when a block is removed + Tathagata Das + 2014-05-08 22:34:08 -0700 + Commit: 32868f3, github.com/apache/spark/pull/708 + + MINOR: Removing dead code. + Patrick Wendell + 2014-05-08 22:33:06 -0700 + Commit: 4c60fd1 + + SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfo + Sandeep + 2014-05-08 22:30:17 -0700 + Commit: 7db47c4, github.com/apache/spark/pull/707 + + SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`. + Patrick Wendell + 2014-05-08 22:26:17 -0700 + Commit: 06b15ba, github.com/apache/spark/pull/704 + + [SPARK-1631] Correctly set the Yarn app name when launching the AM. + Marcelo Vanzin + 2014-05-08 20:46:11 -0700 + Commit: 3f779d8, github.com/apache/spark/pull/539 + + [SPARK-1755] Respect SparkSubmit --name on YARN + Andrew Or + 2014-05-08 20:45:29 -0700 + Commit: 8b78412, github.com/apache/spark/pull/699 + + Include the sbin/spark-config.sh in spark-executor + Bouke van der Bijl + 2014-05-08 20:43:37 -0700 + Commit: 2fd2752, github.com/apache/spark/pull/651 + + Bug fix of sparse vector conversion + Funes , funes + 2014-05-08 17:54:10 -0700 + Commit: 191279c, github.com/apache/spark/pull/661 + + [SPARK-1157][MLlib] Bug fix: lossHistory should exclude rejection steps, and remove miniBatch + DB Tsai + 2014-05-08 17:53:22 -0700 + Commit: 910a13b, github.com/apache/spark/pull/582 + + MLlib documentation fix + DB Tsai + 2014-05-08 17:52:32 -0700 + Commit: d38febe, github.com/apache/spark/pull/703 + + [SPARK-1754] [SQL] Add missing arithmetic DSL operations. + Takuya UESHIN + 2014-05-08 15:31:47 -0700 + Commit: 322b180, github.com/apache/spark/pull/689 + + Fixing typo in als.py + Evan Sparks + 2014-05-08 13:07:30 -0700 + Commit: 5c5e7d5, github.com/apache/spark/pull/696 + + [SPARK-1745] Move interrupted flag from TaskContext constructor (minor) + Andrew Or + 2014-05-08 12:13:07 -0700 + Commit: c3f8b78, github.com/apache/spark/pull/675 + + SPARK-1565, update examples to be used with spark-submit script. + Prashant Sharma + 2014-05-08 10:23:05 -0700 + Commit: 44dd57f, github.com/apache/spark/pull/552 + + [SQL] Improve SparkSQL Aggregates + Michael Armbrust + 2014-05-08 01:08:43 -0400 + Commit: 19c8fb0, github.com/apache/spark/pull/683 + + Use numpy directly for matrix multiply. + Evan Sparks + 2014-05-08 00:24:36 -0400 + Commit: 6ed7e2c, github.com/apache/spark/pull/687 + + SPARK-1668: Add implicit preference as an option to examples/MovieLensALS + Sandeep + 2014-05-08 00:15:05 -0400 + Commit: 108c4c1, github.com/apache/spark/pull/597 + + SPARK-1544 Add support for deep decision trees. + Manish Amde , manishamde , Evan Sparks + 2014-05-07 17:08:38 -0700 + Commit: f269b01, github.com/apache/spark/pull/475 + + Update GradientDescentSuite.scala + baishuo(白硕) + 2014-05-07 16:02:55 -0700 + Commit: 0c19bb1, github.com/apache/spark/pull/588 + + [SPARK-1743][MLLIB] add loadLibSVMFile and saveAsLibSVMFile to pyspark + Xiangrui Meng + 2014-05-07 16:01:11 -0700 + Commit: 3188553, github.com/apache/spark/pull/672 + + SPARK-1569 Spark on Yarn, authentication broken by pr299 + Thomas Graves + 2014-05-07 15:51:53 -0700 + Commit: 4bec84b, github.com/apache/spark/pull/649 + + [SPARK-1688] Propagate PySpark worker stderr to driver + Andrew Or + 2014-05-07 14:35:22 -0700 + Commit: 5200872, github.com/apache/spark/pull/603 + + Typo fix: fetchting -> fetching + Andrew Ash + 2014-05-07 17:24:49 -0400 + Commit: d00981a, github.com/apache/spark/pull/680 + + Nicer logging for SecurityManager startup + Andrew Ash + 2014-05-07 17:24:12 -0400 + Commit: 7f6f4a1, github.com/apache/spark/pull/678 + + [SQL] Fix Performance Issue in data type casting + Cheng Hao + 2014-05-07 16:54:58 -0400 + Commit: ca43186, github.com/apache/spark/pull/679 + + SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions + Aaron Davidson + 2014-05-07 09:48:31 -0700 + Commit: 3308722, github.com/apache/spark/pull/640 + + [SPARK-1460] Returning SchemaRDD instead of normal RDD on Set operations... + Kan Zhang + 2014-05-07 09:41:31 -0700 + Commit: 967635a, github.com/apache/spark/pull/448 + + [WIP][Spark-SQL] Optimize the Constant Folding for Expression + Cheng Hao , Michael Armbrust + 2014-05-07 03:37:12 -0400 + Commit: 3eb53bd, github.com/apache/spark/pull/482 + + SPARK-1746: Support setting SPARK_JAVA_OPTS on executors for backwards compatibility + Patrick Wendell + 2014-05-07 00:11:05 -0700 + Commit: 913a0a9, github.com/apache/spark/pull/676 + + [HOTFIX] SPARK-1637: There are some Streaming examples added after the PR #571 was last updated. + Sandeep + 2014-05-06 21:55:05 -0700 + Commit: fdae095, github.com/apache/spark/pull/673 + + Proposal: clarify Scala programming guide on caching ... + Ethan Jewett + 2014-05-06 20:50:08 -0700 + Commit: 48ba3b8, github.com/apache/spark/pull/668 + + SPARK-1727. Correct small compile errors, typos, and markdown issues in (primarly) MLlib docs + Sean Owen + 2014-05-06 20:07:22 -0700 + Commit: 25ad8f9, github.com/apache/spark/pull/653 + + SPARK-1637: Clean up examples for 1.0 + Sandeep + 2014-05-06 17:27:52 -0700 + Commit: a000b5c, github.com/apache/spark/pull/571 + + SPARK-1737: Warn rather than fail when Java 7+ is used to create distributions + Patrick Wendell + 2014-05-06 15:41:46 -0700 + Commit: 39b8b14, github.com/apache/spark/pull/669 + + [SPARK-1549] Add Python support to spark-submit + Matei Zaharia + 2014-05-06 15:12:35 -0700 + Commit: 951a5d9, github.com/apache/spark/pull/664 + + SPARK-1734: spark-submit throws an exception: Exception in thread "main"... + witgo + 2014-05-06 14:17:39 -0700 + Commit: ec09acd, github.com/apache/spark/pull/665 + + [SPARK-1685] Cancel retryTimer on restart of Worker or AppClient + Mark Hamstra + 2014-05-06 12:53:39 -0700 + Commit: fbfe69d, github.com/apache/spark/pull/602 + + Fix two download suggestions in the docs: + Patrick Wendell + 2014-05-06 12:07:46 -0700 + Commit: 7b978c1, github.com/apache/spark/pull/662 + + SPARK-1474: Spark on yarn assembly doesn't include AmIpFilter + Thomas Graves + 2014-05-06 12:00:09 -0700 + Commit: 1e82990, github.com/apache/spark/pull/406 + + Update OpenHashSet.scala + ArcherShao + 2014-05-06 10:12:59 -0700 + Commit: 0a5a468, github.com/apache/spark/pull/667 + + [SQL] SPARK-1732 - Support for null primitive values. + Michael Armbrust + 2014-05-05 22:59:42 -0700 + Commit: 3c64750, github.com/apache/spark/pull/658 + + [SPARK-1735] Add the missing special profiles to make-distribution.sh + Andrew Or + 2014-05-05 22:14:47 -0700 + Commit: a2262cd, github.com/apache/spark/pull/660 + + [SPARK-1678][SPARK-1679] In-memory compression bug fix and made compression configurable, disabled by default + Cheng Lian + 2014-05-05 19:38:59 -0700 + Commit: 6d721c5, github.com/apache/spark/pull/608 + + [SPARK-1594][MLLIB] Cleaning up MLlib APIs and guide + Xiangrui Meng + 2014-05-05 18:32:54 -0700 + Commit: 98750a7, github.com/apache/spark/pull/524 + + Expose SparkListeners and relevant classes as DeveloperApi + Andrew Or + 2014-05-05 18:32:14 -0700 + Commit: ea10b31, github.com/apache/spark/pull/648 + + SPARK-1728. JavaRDDLike.mapPartitionsWithIndex requires ClassTag + Sandy Ryza + 2014-05-05 18:26:34 -0700 + Commit: 8e724dc, github.com/apache/spark/pull/657 + + [SPARK-1681] Include datanucleus jars in Spark Hive distribution + Andrew Or + 2014-05-05 16:28:07 -0700 + Commit: cf0a8f0, github.com/apache/spark/pull/610 + + [SPARK-1504], [SPARK-1505], [SPARK-1558] Updated Spark Streaming guide + Tathagata Das + 2014-05-05 15:28:19 -0700 + Commit: a975a19, github.com/apache/spark/pull/652 + + SPARK-1721: Reset the thread classLoader in the Mesos Executor + Bouke van der Bijl + 2014-05-05 11:19:35 -0700 + Commit: 3292e2a, github.com/apache/spark/pull/620 + + SPARK-1556. jets3t dep doesn't update properly with newer Hadoop versions + Sean Owen + 2014-05-05 10:33:49 -0700 + Commit: 73b0cbc, github.com/apache/spark/pull/629 + + Updated doc for spark.closure.serializer to indicate only Java serializer work. + Reynold Xin + 2014-05-05 00:52:06 -0700 + Commit: f2eb070, github.com/apache/spark/pull/642 + + Address SPARK-1717 + msiddalingaiah + 2014-05-04 21:59:10 -0700 + Commit: bb2bb0c, github.com/apache/spark/pull/641 + + SPARK-1710: spark-submit should print better errors than "InvocationTargetException" + Sandeep + 2014-05-04 20:51:53 -0700 + Commit: b48a55a, github.com/apache/spark/pull/630 + + EC2 script should exit with non-zero code on UsageError + Allan Douglas R. de Oliveira + 2014-05-04 20:36:51 -0700 + Commit: bcb9b7f, github.com/apache/spark/pull/638 + + SPARK-1693: Most of the tests throw a java.lang.SecurityException when s... + witgo + 2014-05-04 17:48:52 -0700 + Commit: d940e4c, github.com/apache/spark/pull/628 + + SPARK-1629. Addendum: Depend on commons lang3 (already used by tachyon) as it's used in ReplSuite, and return to use lang3 utility in Utils.scala + Sean Owen + 2014-05-04 17:43:28 -0700 + Commit: f504157, github.com/apache/spark/pull/635 + + SPARK-1703 Warn users if Spark is run on JRE6 but compiled with JDK7. + Patrick Wendell + 2014-05-04 12:22:23 -0700 + Commit: 0c98a8f, github.com/apache/spark/pull/627 + + SPARK-1663. (Addendum) Fix signature of one version of JavaPairRDDStream.reduceByKeyAndWindow() + Sean Owen + 2014-05-04 11:55:29 -0700 + Commit: 0088ced, github.com/apache/spark/pull/633 + + SPARK-1658: Correctly identify if maven is installed and working + Rahul Singhal + 2014-05-04 11:08:39 -0700 + Commit: e97a2e6, github.com/apache/spark/pull/580 + + The default version of yarn is equal to the hadoop version + witgo + 2014-05-03 23:32:12 -0700 + Commit: fb05432, github.com/apache/spark/pull/626 + + Whitelist Hive Tests + Michael Armbrust + 2014-05-03 23:13:51 -0700 + Commit: 92b2902, github.com/apache/spark/pull/596 + + [SQL] Better logging when applying rules. + Michael Armbrust + 2014-05-03 18:38:44 -0700 + Commit: b295714, github.com/apache/spark/pull/616 + + EC2 configurable workers + Allan Douglas R. de Oliveira + 2014-05-03 16:52:19 -0700 + Commit: 4669a84, github.com/apache/spark/pull/612 + + SPARK-1689 AppClient should indicate app is dead() when removed + Aaron Davidson + 2014-05-03 13:27:10 -0700 + Commit: 34719ba, github.com/apache/spark/pull/605 + + [Bugfix] Tachyon file cleanup logical error + Cheng Lian + 2014-05-03 13:23:52 -0700 + Commit: ce72c72, github.com/apache/spark/pull/575 + + SPARK-1663. Corrections for several compile errors in streaming code examples, and updates to follow API changes + Sean Owen + 2014-05-03 12:31:31 -0700 + Commit: 11d5494, github.com/apache/spark/pull/589 + + [WIP] SPARK-1676: Cache Hadoop UGIs by default to prevent FileSystem leak + Thomas Graves + 2014-05-03 10:59:05 -0700 + Commit: 3d0a02d, github.com/apache/spark/pull/621 + + Update SchemaRDD.scala + ArcherShao + 2014-05-03 00:17:36 -0700 + Commit: 9347565, github.com/apache/spark/pull/619 + + SPARK-1700: Close socket file descriptors on task completion + Aaron Davidson + 2014-05-02 23:55:13 -0700 + Commit: 0a14421, github.com/apache/spark/pull/623 + + SPARK-1492. Update Spark YARN docs to use spark-submit + Sandy Ryza + 2014-05-02 21:42:31 -0700 + Commit: 2b961d8, github.com/apache/spark/pull/601 + + delete no use var + wangfei + 2014-05-02 21:34:54 -0700 + Commit: 4bf24f7, github.com/apache/spark/pull/613 + + SPARK-1695: java8-tests compiler error: package com.google.common.co... + witgo + 2014-05-02 12:40:27 -0700 + Commit: f25ebed, github.com/apache/spark/pull/611 + + Add tests for FileLogger, EventLoggingListener, and ReplayListenerBus + Andrew Or + 2014-05-01 21:42:06 -0700 + Commit: 394d8cb, github.com/apache/spark/pull/591 + + SPARK-1659: improvements spark-submit usage + witgo + 2014-05-01 21:39:40 -0700 + Commit: 40cf6d3, github.com/apache/spark/pull/581 + + fix the spelling mistake + wangfei + 2014-05-01 21:37:22 -0700 + Commit: 55c760f, github.com/apache/spark/pull/614 + + [SQL] SPARK-1661 - Fix regex_serde test + Michael Armbrust + 2014-05-01 21:32:43 -0700 + Commit: a43d9c1, github.com/apache/spark/pull/595 + + SPARK-1691: Support quoted arguments inside of spark-submit. + Patrick Wendell + 2014-05-01 01:15:51 -0700 + Commit: 98b6559, github.com/apache/spark/pull/609 + + Fix SPARK-1629: Spark should inline use of commons-lang `SystemUtils.IS_... + witgo + 2014-04-30 09:49:45 -0700 + Commit: 55100da, github.com/apache/spark/pull/569 + + SPARK-1004. PySpark on YARN + Sandy Ryza + 2014-04-29 23:24:34 -0700 + Commit: ff5be9a, github.com/apache/spark/pull/30 + + Handle the vals that never used + WangTao + 2014-04-29 22:07:20 -0700 + Commit: 7025dda, github.com/apache/spark/pull/565 + + Args for worker rather than master + Chen Chao + 2014-04-29 22:05:40 -0700 + Commit: b3d2ab6, github.com/apache/spark/pull/587 + + [SPARK-1646] Micro-optimisation of ALS + Tor Myklebust + 2014-04-29 22:04:34 -0700 + Commit: 5c0cd5c, github.com/apache/spark/pull/568 + + [SPARK-1674] fix interrupted system call error in pyspark's RDD.pipe + Xiangrui Meng + 2014-04-29 18:06:45 -0700 + Commit: d33df1c, github.com/apache/spark/pull/594 + + SPARK-1588. Restore SPARK_YARN_USER_ENV and SPARK_JAVA_OPTS for YARN. + Sandy Ryza + 2014-04-29 12:54:02 -0700 + Commit: bf8d0aa, github.com/apache/spark/pull/586 + + SPARK-1509: add zipWithIndex zipWithUniqueId methods to java api + witgo + 2014-04-29 11:30:47 -0700 + Commit: 7d15058, github.com/apache/spark/pull/423 + + SPARK-1557 Set permissions on event log files/directories + Thomas Graves + 2014-04-29 09:19:48 -0500 + Commit: 8db0f7e, github.com/apache/spark/pull/538 + + HOTFIX: minor change to release script + Patrick Wendell + 2014-04-29 00:59:38 -0700 + Commit: 9a1184a + + HOTFIX: minor change to release script + Patrick Wendell + 2014-04-29 00:53:32 -0700 + Commit: f04bcaf + + [SPARK-1636][MLLIB] Move main methods to examples + Xiangrui Meng + 2014-04-29 00:41:03 -0700 + Commit: 3f38334, github.com/apache/spark/pull/584 + + Minor fix to python table caching API. + Michael Armbrust + 2014-04-29 00:36:15 -0700 + Commit: 497be3c, github.com/apache/spark/pull/585 + + HOTFIX: Bug in release script + Patrick Wendell + 2014-04-29 00:10:17 -0700 + Commit: 719c8bc + + Improved build configuration + witgo + 2014-04-28 22:50:51 -0700 + Commit: 030f2c2, github.com/apache/spark/pull/480 + + SPARK-1652: Remove incorrect deprecation warning in spark-submit + Patrick Wendell + 2014-04-28 18:14:59 -0700 + Commit: 9f7a095, github.com/apache/spark/pull/578 + + SPARK-1654 and SPARK-1653: Fixes in spark-submit. + Patrick Wendell + 2014-04-28 17:29:22 -0700 + Commit: 949e393, github.com/apache/spark/pull/576 + + SPARK-1652: Spark submit should fail gracefully if YARN not enabled + Patrick Wendell + 2014-04-28 17:26:57 -0700 + Commit: cae054a, github.com/apache/spark/pull/579 + + Changes to dev release script + Patrick Wendell + 2014-04-28 13:58:42 -0700 + Commit: 8421034 + + [SPARK-1633][Streaming] Java API unit test and example for custom streaming receiver in Java + Tathagata Das + 2014-04-28 13:58:09 -0700 + Commit: 1d84964, github.com/apache/spark/pull/558 + + [SQL]Append some missing types for HiveUDF + Cheng Hao + 2014-04-27 23:59:42 -0700 + Commit: f735884, github.com/apache/spark/pull/459 + + Update the import package name for TestHive in sbt shell + Cheng Hao + 2014-04-27 23:57:29 -0700 + Commit: ea01aff, github.com/apache/spark/pull/574 + + Fix SPARK-1609: Executor fails to start when Command.extraJavaOptions contains multiple Java options + witgo + 2014-04-27 19:41:02 -0700 + Commit: 71f4d26, github.com/apache/spark/pull/547 + + SPARK-1145: Memory mapping with many small blocks can cause JVM allocation failures + Patrick Wendell + 2014-04-27 17:40:56 -0700 + Commit: 6b3c6e5, github.com/apache/spark/pull/43 + + HOTFIX: Minor patch to merge script. + Patrick Wendell + 2014-04-27 15:45:17 -0700 + Commit: 3d9fb09 + + SPARK-1651: Delete existing deployment directory + Rahul Singhal + 2014-04-27 15:50:48 -0700 + Commit: eefb90d, github.com/apache/spark/pull/573 + + SPARK-1648 Support closing JIRA's as part of merge script. + Patrick Wendell + 2014-04-27 15:41:57 -0700 + Commit: fe65bee, github.com/apache/spark/pull/570 + + SPARK-1650: Correctly identify maven project version + Rahul Singhal + 2014-04-27 15:17:06 -0700 + Commit: 7b2527d, github.com/apache/spark/pull/572 + + SPARK-1606: Infer user application arguments instead of requiring --arg. + Patrick Wendell + 2014-04-26 19:24:29 -0700 + Commit: aa9a7f5, github.com/apache/spark/pull/563 + + SPARK-1467: Make StorageLevel.apply() factory methods Developer APIs + Sandeep + 2014-04-26 19:04:33 -0700 + Commit: 762af4e, github.com/apache/spark/pull/551 + + [SPARK-1608] [SQL] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. + Takuya UESHIN + 2014-04-26 14:39:54 -0700 + Commit: 8e37ed6, github.com/apache/spark/pull/532 + + add note of how to support table with more than 22 fields + wangfei + 2014-04-26 14:38:42 -0700 + Commit: e6e44e4, github.com/apache/spark/pull/564 + + [Spark-1382] Fix NPE in DStream.slice (updated version of #365) + zsxwing , Tathagata Das + 2014-04-25 19:04:34 -0700 + Commit: 058797c, github.com/apache/spark/pull/562 + + SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyM... + Sandy Ryza + 2014-04-25 17:55:04 -0700 + Commit: 87cf35c, github.com/apache/spark/pull/559 + + SPARK-1235: manage the DAGScheduler EventProcessActor with supervisor and refactor the DAGScheduler with Akka + CodingCat , Xiangrui Meng , Nan Zhu + 2014-04-25 16:04:48 -0700 + Commit: 027f1b8, github.com/apache/spark/pull/186 + + SPARK-1607. HOTFIX: Fix syntax adapting Int result to Short + Sean Owen + 2014-04-25 14:17:38 -0700 + Commit: df6d814, github.com/apache/spark/pull/556 + + Update KafkaWordCount.scala + baishuo(白硕) + 2014-04-25 13:18:49 -0700 + Commit: 8aaef5c, github.com/apache/spark/pull/523 + + Delete the val that never used + WangTao + 2014-04-25 11:47:01 -0700 + Commit: 25a276d, github.com/apache/spark/pull/553 + + SPARK-1621 Upgrade Chill to 0.3.6 + Matei Zaharia + 2014-04-25 11:12:41 -0700 + Commit: a24d918, github.com/apache/spark/pull/543 + + SPARK-1619 Launch spark-shell with spark-submit + Patrick Wendell + 2014-04-24 23:59:16 -0700 + Commit: dc3b640, github.com/apache/spark/pull/542 + + SPARK-1607. Replace octal literals, removed in Scala 2.11, with hex literals + Sean Owen + 2014-04-24 23:34:00 -0700 + Commit: 6e101f1, github.com/apache/spark/pull/529 + + Call correct stop(). + Aaron Davidson + 2014-04-24 23:22:03 -0700 + Commit: 45ad7f0, github.com/apache/spark/pull/527 + + SPARK-1242 Add aggregate to python rdd + Holden Karau + 2014-04-24 23:07:54 -0700 + Commit: e03bc37, github.com/apache/spark/pull/139 + + Fix [SPARK-1078]: Remove the Unnecessary lift-json dependency + Sandeep + 2014-04-24 21:51:52 -0700 + Commit: 095b518, github.com/apache/spark/pull/536 + + [Typo] In the maven docs: chd -> cdh + Andrew Or + 2014-04-24 21:51:17 -0700 + Commit: 06e82d9, github.com/apache/spark/pull/548 + + Generalize pattern for planning hash joins. + Michael Armbrust + 2014-04-24 21:42:33 -0700 + Commit: 86ff8b1, github.com/apache/spark/pull/418 + + [SPARK-1617] and [SPARK-1618] Improvements to streaming ui and bug fix to socket receiver + Tathagata Das + 2014-04-24 21:34:37 -0700 + Commit: cd12dd9, github.com/apache/spark/pull/540 + + SPARK-1586 Windows build fixes + Mridul Muralidharan + 2014-04-24 20:48:33 -0700 + Commit: 968c018, github.com/apache/spark/pull/505 + + SPARK-1584: Upgrade Flume dependency to 1.4.0 + tmalaska + 2014-04-24 20:31:17 -0700 + Commit: d5c6ae6, github.com/apache/spark/pull/507 + + [SPARK-986]: Job cancelation for PySpark + Ahir Reddy + 2014-04-24 20:21:10 -0700 + Commit: e53eb4f, github.com/apache/spark/pull/541 + + [SPARK-1615] Synchronize accesses to the LiveListenerBus' event queue + Andrew Or + 2014-04-24 20:18:15 -0700 + Commit: ee6f7e2, github.com/apache/spark/pull/544 + + [SPARK-1510] Spark Streaming metrics source for metrics system + jerryshao , Tathagata Das + 2014-04-24 18:56:57 -0700 + Commit: 80429f3, github.com/apache/spark/pull/545 + + Spark 1489 Fix the HistoryServer view acls + Thomas Graves + 2014-04-24 18:38:10 -0700 + Commit: 44da5ab, github.com/apache/spark/pull/509 + + [SQL] Add support for parsing indexing into arrays in SQL. + Michael Armbrust + 2014-04-24 18:21:00 -0700 + Commit: 4660991, github.com/apache/spark/pull/518 + + [SPARK-1592][streaming] Automatically remove streaming input blocks + Tathagata Das + 2014-04-24 18:18:22 -0700 + Commit: 526a518, github.com/apache/spark/pull/512 + + SPARK-1438 RDD.sample() make seed param optional + Arun Ramakrishnan + 2014-04-24 17:27:16 -0700 + Commit: 35e3d19, github.com/apache/spark/pull/477 + + SPARK-1104: kill Process in workerThread of ExecutorRunner + CodingCat + 2014-04-24 15:55:18 -0700 + Commit: f99af85, github.com/apache/spark/pull/35 + + Fix Scala Style + Sandeep + 2014-04-24 15:07:23 -0700 + Commit: a03ac22, github.com/apache/spark/pull/531 + + SPARK-1494 Don't initialize classes loaded by MIMA excludes, attempt 2 + Michael Armbrust + 2014-04-24 14:54:01 -0700 + Commit: c5c1916, github.com/apache/spark/pull/526 + + Spark 1490 Add kerberos support to the HistoryServer + Thomas Graves + 2014-04-24 11:15:12 -0700 + Commit: bd37509, github.com/apache/spark/pull/513 + + SPARK-1611: Fix incorrect initialization order in AppendOnlyMap + zsxwing + 2014-04-24 11:13:40 -0700 + Commit: 78a49b2, github.com/apache/spark/pull/534 + + SPARK-1488. Squash more language feature warnings in new commits by importing implicitConversion + Sean Owen + 2014-04-24 10:06:18 -0700 + Commit: 6338a93, github.com/apache/spark/pull/528 + + Small changes to release script + Patrick Wendell + 2014-04-24 09:59:44 -0700 + Commit: faeb761 + + [SPARK-1610] [SQL] Fix Cast to use exact type value when cast from BooleanType to NumericTy... + Takuya UESHIN + 2014-04-24 09:57:28 -0700 + Commit: 27b2821, github.com/apache/spark/pull/533 + + SPARK-1601 & SPARK-1602: two bug fixes related to cancellation + Reynold Xin + 2014-04-24 00:27:45 -0700 + Commit: 1fdf659, github.com/apache/spark/pull/521 + + SPARK-1587 Fix thread leak + Mridul Muralidharan + 2014-04-23 23:20:55 -0700 + Commit: dd681f5, github.com/apache/spark/pull/504 + + [Fix #79] Replace Breakable For Loops By While Loops + Sandeep + 2014-04-23 22:47:59 -0700 + Commit: bb68f47, github.com/apache/spark/pull/503 + + SPARK-1589: Fix the incorrect compare + zsxwing + 2014-04-23 22:36:02 -0700 + Commit: 6ab7578, github.com/apache/spark/pull/508 + + Mark all fields of EdgePartition, Graph, and GraphOps transient + Ankur Dave + 2014-04-23 22:01:13 -0700 + Commit: 1d6abe3, github.com/apache/spark/pull/520 + + Update Java api for setJobGroup with interruptOnCancel + Aaron Davidson + 2014-04-23 22:00:22 -0700 + Commit: d485eec, github.com/apache/spark/pull/522 + + [Hot Fix #469] Fix flaky test in SparkListenerSuite + Andrew Or + 2014-04-23 21:59:33 -0700 + Commit: 4b2bab1, github.com/apache/spark/pull/516 + + [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions. + Matei Zaharia + 2014-04-23 17:03:54 -0700 + Commit: 640f9a0, github.com/apache/spark/pull/487 + + SPARK-1582 Invoke Thread.interrupt() when cancelling jobs + Aaron Davidson + 2014-04-23 16:52:49 -0700 + Commit: 432201c, github.com/apache/spark/pull/498 + + Honor default fs name when initializing event logger. + Marcelo Vanzin + 2014-04-23 14:47:38 -0700 + Commit: dd1b7a6, github.com/apache/spark/pull/450 + + SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent + Aaron Davidson + 2014-04-23 14:46:30 -0700 + Commit: a967b00, github.com/apache/spark/pull/486 + + SPARK-1583: Fix a bug that using java.util.HashMap by mistake + zsxwing + 2014-04-23 14:12:20 -0700 + Commit: a664606, github.com/apache/spark/pull/500 + + SPARK-1119 and other build improvements + Patrick Wendell + 2014-04-23 10:19:32 -0700 + Commit: cd4ed29, github.com/apache/spark/pull/502 + + [SQL] SPARK-1571 Mistake in java example code + Michael Armbrust + 2014-04-22 22:19:32 -0700 + Commit: 39f85e0, github.com/apache/spark/pull/496 + + SPARK-1494 Don't initialize classes loaded by MIMA excludes. + Michael Armbrust + 2014-04-22 21:56:15 -0700 + Commit: 8e95081, github.com/apache/spark/pull/494 + + SPARK-1562 Fix visibility / annotation of Spark SQL APIs + Michael Armbrust + 2014-04-22 20:02:33 -0700 + Commit: aa77f8a, github.com/apache/spark/pull/489 + + [FIX: SPARK-1376] use --arg instead of --args in SparkSubmit to avoid warning messages + Xiangrui Meng + 2014-04-22 19:38:27 -0700 + Commit: 662c860, github.com/apache/spark/pull/485 + + [streaming][SPARK-1578] Removed requirement for TTL in StreamingContext. + Tathagata Das + 2014-04-22 19:35:13 -0700 + Commit: f3d19a9, github.com/apache/spark/pull/491 + + [Spark-1538] Fix SparkUI incorrectly hiding persisted RDDs + Andrew Or + 2014-04-22 19:24:03 -0700 + Commit: 2de5738, github.com/apache/spark/pull/469 + + Assorted clean-up for Spark-on-YARN. + Patrick Wendell + 2014-04-22 19:22:06 -0700 + Commit: 995fdc9, github.com/apache/spark/pull/488 + + [SPARK-1570] Fix classloading in JavaSQLContext.applySchema + Kan Zhang + 2014-04-22 15:05:12 -0700 + Commit: ea8cea8, github.com/apache/spark/pull/484 + + Fix compilation on Hadoop 2.4.x. + Marcelo Vanzin + 2014-04-22 14:28:41 -0700 + Commit: 0ea0b1a, github.com/apache/spark/pull/483 + + [Fix #204] Eliminate delay between binding and log checking + Andrew Or + 2014-04-22 14:27:49 -0700 + Commit: 745e496, github.com/apache/spark/pull/441 + + [SPARK-1506][MLLIB] Documentation improvements for MLlib 1.0 + Xiangrui Meng + 2014-04-22 11:20:47 -0700 + Commit: 26d35f3, github.com/apache/spark/pull/422 + + [SPARK-1281] Improve partitioning in ALS + Tor Myklebust + 2014-04-22 11:07:30 -0700 + Commit: bf9d49b, github.com/apache/spark/pull/407 + + fix bugs of dot in python + Xusen Yin + 2014-04-22 11:06:18 -0700 + Commit: c919798, github.com/apache/spark/pull/463 + + [SPARK-1560]: Updated Pyrolite Dependency to be Java 6 compatible + Ahir Reddy + 2014-04-22 09:44:41 -0700 + Commit: 0f87e6a, github.com/apache/spark/pull/479 + + [HOTFIX] SPARK-1399: remove outdated comments + CodingCat + 2014-04-22 09:43:13 -0700 + Commit: 87de290, github.com/apache/spark/pull/474 + + SPARK-1496: Have jarOfClass return Option[String] + Patrick Wendell + 2014-04-22 00:42:16 -0700 + Commit: 83084d3, github.com/apache/spark/pull/438 + + [SPARK-1459] Use local path (and not complete URL) when opening local lo... + Marcelo Vanzin + 2014-04-21 23:10:53 -0700 + Commit: ac164b7, github.com/apache/spark/pull/375 + + [Fix #274] Document + fix annotation usages + Andrew Or + 2014-04-21 22:24:44 -0700 + Commit: b3e5366, github.com/apache/spark/pull/470 + + [SPARK-1439, SPARK-1440] Generate unified Scaladoc across projects and Javadocs + Matei Zaharia + 2014-04-21 21:57:40 -0700 + Commit: fc78384, github.com/apache/spark/pull/457 + + [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API [WIP] + Tathagata Das + 2014-04-21 19:04:49 -0700 + Commit: 04c37b6, github.com/apache/spark/pull/300 + + Dev script: include RC name in git tag + Patrick Wendell + 2014-04-21 14:21:17 -0700 + Commit: 5a5b334 + + SPARK-1399: show stage failure reason in UI + CodingCat , Nan Zhu + 2014-04-21 14:10:23 -0700 + Commit: 43e4a29, github.com/apache/spark/pull/421 + + SPARK-1539: RDDPage.scala contains RddPage class + Xiangrui Meng + 2014-04-21 12:48:02 -0700 + Commit: b7df31e, github.com/apache/spark/pull/454 + + [Hot Fix] Ignore org.apache.spark.ui.UISuite tests + Andrew Or + 2014-04-21 12:37:43 -0700 + Commit: af46f1f, github.com/apache/spark/pull/466 + + Clean up and simplify Spark configuration + Patrick Wendell + 2014-04-21 10:26:33 -0700 + Commit: fb98488, github.com/apache/spark/pull/299 + + REPL cleanup. + Michael Armbrust + 2014-04-19 17:32:24 -0700 + Commit: 3a390bf, github.com/apache/spark/pull/451 + + [SPARK-1535] ALS: Avoid the garbage-creating ctor of DoubleMatrix + Tor Myklebust + 2014-04-19 15:10:18 -0700 + Commit: 25fc318, github.com/apache/spark/pull/442 + + Add insertInto and saveAsTable to Python API. + Michael Armbrust + 2014-04-19 15:08:54 -0700 + Commit: 10d0421, github.com/apache/spark/pull/447 + + Use scala deprecation instead of java. + Michael Armbrust + 2014-04-19 15:06:04 -0700 + Commit: 5d0f58b, github.com/apache/spark/pull/452 + + README update + Reynold Xin + 2014-04-18 22:34:39 -0700 + Commit: 28238c8, github.com/apache/spark/pull/443 + + SPARK-1482: Fix potential resource leaks in saveAsHadoopDataset and save... + zsxwing + 2014-04-18 17:49:22 -0700 + Commit: 2089e0e, github.com/apache/spark/pull/400 + + SPARK-1456 Remove view bounds on Ordered in favor of a context bound on Ordering. + Michael Armbrust + 2014-04-18 12:04:13 -0700 + Commit: c399baa, github.com/apache/spark/pull/410 + + Fixed broken pyspark shell. + Reynold Xin + 2014-04-18 10:10:13 -0700 + Commit: 81a152c, github.com/apache/spark/pull/444 + + SPARK-1523: improve the readability of code in AkkaUtil + CodingCat + 2014-04-18 10:05:00 -0700 + Commit: 3c7a9ba, github.com/apache/spark/pull/434 + + SPARK-1357 (addendum). More Experimental items in MLlib + Sean Owen + 2014-04-18 10:04:02 -0700 + Commit: 8aa1f4c, github.com/apache/spark/pull/372 + + [SPARK-1520] remove fastutil from dependencies + Xiangrui Meng + 2014-04-18 10:03:15 -0700 + Commit: aa17f02, github.com/apache/spark/pull/437 + + Reuses Row object in ExistingRdd.productToRowRdd() + Cheng Lian + 2014-04-18 10:02:27 -0700 + Commit: 89f4743, github.com/apache/spark/pull/432 + + SPARK-1483: Rename minSplits to minPartitions in public APIs + CodingCat + 2014-04-18 10:01:16 -0700 + Commit: e31c8ff, github.com/apache/spark/pull/430 + + HOTFIX: Ignore streaming UI test + Patrick Wendell + 2014-04-17 17:33:24 -0700 + Commit: 7863ecc, github.com/apache/spark/pull/440 + + FIX: Don't build Hive in assembly unless running Hive tests. + Patrick Wendell + 2014-04-17 17:24:00 -0700 + Commit: 6c746ba, github.com/apache/spark/pull/439 + + SPARK-1408 Modify Spark on Yarn to point to the history server when app ... + Thomas Graves + 2014-04-17 16:36:37 -0500 + Commit: 0058b5d, github.com/apache/spark/pull/362 + + [SPARK-1395] Allow "local:" URIs to work on Yarn. + Marcelo Vanzin + 2014-04-17 10:29:38 -0500 + Commit: 6904750, github.com/apache/spark/pull/303 + + [python alternative] pyspark require Python2, failing if system default is Py3 from shell.py + AbhishekKr + 2014-04-16 19:05:40 -0700 + Commit: bb76eae, github.com/apache/spark/pull/399 + + SPARK-1462: Examples of ML algorithms are using deprecated APIs + Sandeep + 2014-04-16 18:23:07 -0700 + Commit: 6ad4c54, github.com/apache/spark/pull/416 + + Include stack trace for exceptions thrown by user code. + Michael Armbrust + 2014-04-16 18:12:56 -0700 + Commit: d4916a8, github.com/apache/spark/pull/409 + + Update ReducedWindowedDStream.scala + baishuo(白硕) + 2014-04-16 18:08:11 -0700 + Commit: 07b7ad3, github.com/apache/spark/pull/425 + + misleading task number of groupByKey + Chen Chao + 2014-04-16 17:58:42 -0700 + Commit: 9c40b9e, github.com/apache/spark/pull/403 + + Fixing a race condition in event listener unit test + Kan Zhang + 2014-04-16 17:39:11 -0700 + Commit: 38877cc, github.com/apache/spark/pull/401 + + remove unnecessary brace and semicolon in 'putBlockInfo.synchronize' block + Chen Chao + 2014-04-16 17:30:01 -0700 + Commit: 016a877, github.com/apache/spark/pull/411 + + SPARK-1329: Create pid2vid with correct number of partitions + Ankur Dave + 2014-04-16 17:16:55 -0700 + Commit: 17d3234, github.com/apache/spark/pull/368 + + Rebuild routing table after Graph.reverse + Ankur Dave + 2014-04-16 17:15:50 -0700 + Commit: 235a47c, github.com/apache/spark/pull/431 + + Add clean to build + Patrick Wendell + 2014-04-16 16:32:34 -0700 + Commit: 987760e + + [SPARK-1511] use Files.move instead of renameTo in TestUtils.scala + Ye Xianjin + 2014-04-16 14:56:22 -0700 + Commit: 10b1c59, github.com/apache/spark/pull/427 + + SPARK-1465: Spark compilation is broken with the latest hadoop-2.4.0 release + xuan , xuan + 2014-04-16 14:41:22 -0500 + Commit: 725925c, github.com/apache/spark/pull/396 + + SPARK-1469: Scheduler mode should accept lower-case definitions and have... + Sandeep + 2014-04-16 09:58:57 -0700 + Commit: e269c24, github.com/apache/spark/pull/388 + + Minor addition to SPARK-1497 + Patrick Wendell + 2014-04-16 09:43:17 -0700 + Commit: 82349fb + + SPARK-1497. Fix scalastyle warnings in YARN, Hive code + Sean Owen + 2014-04-16 09:34:59 -0700 + Commit: 77f8367, github.com/apache/spark/pull/413 + + SPARK-1310: Start adding k-fold cross validation to MLLib [adds kFold to MLUtils & fixes bug in BernoulliSampler] + Holden Karau + 2014-04-16 09:33:27 -0700 + Commit: c3527a3, github.com/apache/spark/pull/18 + + update spark.default.parallelism + Chen Chao + 2014-04-16 09:14:18 -0700 + Commit: 9edd887, github.com/apache/spark/pull/389 + + Loads test tables when running "sbt hive/console" without HIVE_DEV_HOME + Cheng Lian + 2014-04-16 08:54:34 -0700 + Commit: fec462c, github.com/apache/spark/pull/417 + + Make "spark logo" link refer to "/". + Marcelo Vanzin + 2014-04-16 08:53:01 -0700 + Commit: c0273d8, github.com/apache/spark/pull/408 + + [SPARK-959] Updated SBT from 0.13.1 to 0.13.2 + Cheng Lian + 2014-04-16 08:52:14 -0700 + Commit: 6a10d80, github.com/apache/spark/pull/426 + + [SQL] SPARK-1424 Generalize insertIntoTable functions on SchemaRDDs + Michael Armbrust + 2014-04-15 20:40:40 -0700 + Commit: 273c2fd, github.com/apache/spark/pull/354 + + [WIP] SPARK-1430: Support sparse data in Python MLlib + Matei Zaharia + 2014-04-15 20:33:24 -0700 + Commit: 63ca581, github.com/apache/spark/pull/341 + + [FIX] update sbt-idea to version 1.6.0 + Xiangrui Meng + 2014-04-15 19:37:32 -0700 + Commit: 8517911, github.com/apache/spark/pull/419 + + SPARK-1455: Better isolation for unit tests. + Patrick Wendell + 2014-04-15 19:34:39 -0700 + Commit: 5aaf983, github.com/apache/spark/pull/420 + + Decision Tree documentation for MLlib programming guide + Manish Amde + 2014-04-15 11:14:28 -0700 + Commit: 07d72fe, github.com/apache/spark/pull/402 + + [SPARK-1157][MLlib] L-BFGS Optimizer based on Breeze's implementation. + DB Tsai + 2014-04-15 11:12:47 -0700 + Commit: 6843d63, github.com/apache/spark/pull/353 + + SPARK-1501: Ensure assertions in Graph.apply are asserted. + William Benton + 2014-04-15 10:38:42 -0700 + Commit: 2580a3b, github.com/apache/spark/pull/415 + + SPARK-1426: Make MLlib work with NumPy versions older than 1.7 + Sandeep + 2014-04-15 00:19:43 -0700 + Commit: df36091, github.com/apache/spark/pull/391 + + SPARK-1374: PySpark API for SparkSQL + Ahir Reddy , Michael Armbrust + 2014-04-15 00:07:55 -0700 + Commit: c99bcb7f, github.com/apache/spark/pull/363 + + SPARK-1488. Resolve scalac feature warnings during build + Sean Owen + 2014-04-14 19:50:00 -0700 + Commit: 0247b5c, github.com/apache/spark/pull/404 + + HOTFIX: Use file name and not paths for excludes + Patrick Wendell + 2014-04-14 15:51:54 -0700 + Commit: 268b535 + + [BUGFIX] In-memory columnar storage bug fixes + Cheng Lian , Michael Armbrust + 2014-04-14 15:22:43 -0700 + Commit: 7dbca68, github.com/apache/spark/pull/374 + + [SPARK-1415] Hadoop min split for wholeTextFiles() + Xusen Yin + 2014-04-13 13:18:52 -0700 + Commit: 037fe4d, github.com/apache/spark/pull/376 + + SPARK-1480: Clean up use of classloaders + Patrick Wendell + 2014-04-13 08:58:37 -0700 + Commit: 4bc07ee, github.com/apache/spark/pull/398 + + [SPARK-1403] Move the class loader creation back to where it was in 0.9.0 + Bharath Bhushan + 2014-04-12 20:52:29 -0700 + Commit: ca11919, github.com/apache/spark/pull/322 + + [Fix #204] Update out-dated comments + Andrew Or + 2014-04-12 16:33:38 -0700 + Commit: c2d160f, github.com/apache/spark/pull/381 + + [SPARK-1386] Web UI for Spark Streaming + Tathagata Das , Andrew Or + 2014-04-11 23:33:49 -0700 + Commit: 6aa08c3, github.com/apache/spark/pull/290 + + SPARK-1057 (alternative) Remove fastutil + Sean Owen + 2014-04-11 22:46:47 -0700 + Commit: 165e06a, github.com/apache/spark/pull/266 + + Update WindowedDStream.scala + baishuo(白硕) + 2014-04-11 20:33:42 -0700 + Commit: aa8bb11, github.com/apache/spark/pull/390 + + [WIP] [SPARK-1328] Add vector statistics + Xusen Yin , Xiangrui Meng + 2014-04-11 19:43:22 -0700 + Commit: fdfb45e, github.com/apache/spark/pull/268 + + [FIX] make coalesce test deterministic in RDDSuite + Xiangrui Meng + 2014-04-11 19:41:40 -0700 + Commit: 7038b00, github.com/apache/spark/pull/387 + + HOTFIX: Ignore python metastore files in RAT checks. + Patrick Wendell + 2014-04-11 13:23:21 -0700 + Commit: 6a0f8e3, github.com/apache/spark/pull/393 + + [SPARK-1225, 1241] [MLLIB] Add AreaUnderCurve and BinaryClassificationMetrics + Xiangrui Meng + 2014-04-11 12:06:13 -0700 + Commit: f5ace8d, github.com/apache/spark/pull/364 + + Some clean up in build/docs + Patrick Wendell + 2014-04-11 10:45:27 -0700 + Commit: 98225a6, github.com/apache/spark/pull/382 + + SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken + Thomas Graves + 2014-04-11 13:17:48 +0530 + Commit: 446bb34, github.com/apache/spark/pull/344 + + SPARK-1202: Improvements to task killing in the UI. + Patrick Wendell + 2014-04-10 20:43:56 -0700 + Commit: 44f654e, github.com/apache/spark/pull/386 + + Add Spark v0.9.1 to ec2 launch script and use it as the default + Harvey Feng + 2014-04-10 18:25:54 -0700 + Commit: 7b4203a, github.com/apache/spark/pull/385 + + Set spark.executor.uri from environment variable (needed by Mesos) + Ivan Wick + 2014-04-10 17:49:30 -0700 + Commit: 5cd11d5, github.com/apache/spark/pull/311 + + SPARK-1202 - Add a "cancel" button in the UI for stages + Sundeep Narravula , Sundeep Narravula + 2014-04-10 17:10:11 -0700 + Commit: 2c55783, github.com/apache/spark/pull/246 + + [SQL] Improve column pruning in the optimizer. + Michael Armbrust + 2014-04-10 16:20:33 -0700 + Commit: f99401a, github.com/apache/spark/pull/378 + + Remove Unnecessary Whitespace's + Sandeep + 2014-04-10 15:04:13 -0700 + Commit: 930b70f, github.com/apache/spark/pull/380 + + Update tuning.md + Andrew Ash + 2014-04-10 14:59:58 -0700 + Commit: f046662, github.com/apache/spark/pull/384 + + Revert "SPARK-1433: Upgrade Mesos dependency to 0.17.0" + Patrick Wendell + 2014-04-10 14:43:29 -0700 + Commit: 7b52b66 + + SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining + Sandeep + 2014-04-10 11:17:41 -0700 + Commit: 3bd3129, github.com/apache/spark/pull/356 + + [SPARK-1276] Add a HistoryServer to render persisted UI + Andrew Or + 2014-04-10 10:39:34 -0700 + Commit: 79820fe, github.com/apache/spark/pull/204 + + +Release 1.0.0 + + HOTFIX: Add no-arg SparkContext constructor in Java + Patrick Wendell + 2014-05-25 20:13:32 -0700 + Commit: 18c77cb, github.com/apache/spark/pull/878 + + [SQL] Minor: Introduce SchemaRDD#aggregate() for simple aggregations + Aaron Davidson + 2014-05-25 18:37:44 -0700 + Commit: a3976a2, github.com/apache/spark/pull/874 + + SPARK-1903 Document Spark's network connections + Andrew Ash + 2014-05-25 17:15:47 -0700 + Commit: 5107a6f, github.com/apache/spark/pull/856 + + Fix PEP8 violations in Python mllib. + Reynold Xin + 2014-05-25 17:15:01 -0700 + Commit: 07f34ca, github.com/apache/spark/pull/871 + + Python docstring update for sql.py. + Reynold Xin + 2014-05-25 16:04:17 -0700 + Commit: 8891495, github.com/apache/spark/pull/869 + + Fix PEP8 violations in examples/src/main/python. + Reynold Xin + 2014-05-25 14:48:27 -0700 + Commit: 3368397, github.com/apache/spark/pull/870 + + Added license header for tox.ini. + Reynold Xin + 2014-05-25 01:47:08 -0700 + Commit: 7273bfc + + SPARK-1822: Some minor cleanup work on SchemaRDD.count() + Reynold Xin + 2014-05-25 01:44:49 -0700 + Commit: aeffc20, github.com/apache/spark/pull/868 + + Added PEP8 style configuration file. + Reynold Xin + 2014-05-25 01:32:15 -0700 + Commit: 291567d, github.com/apache/spark/pull/872 + + [SPARK-1822] SchemaRDD.count() should use query optimizer + Kan Zhang + 2014-05-25 00:06:42 -0700 + Commit: 64d0fb5, github.com/apache/spark/pull/841 + + spark-submit: add exec at the end of the script + Colin Patrick Mccabe + 2014-05-24 22:39:27 -0700 + Commit: 7e59335, github.com/apache/spark/pull/858 + + [SPARK-1886] check executor id existence when executor exit + Zhen Peng + 2014-05-24 20:40:19 -0700 + Commit: b5e9686, github.com/apache/spark/pull/827 + + Updated CHANGES.txt + Tathagata Das + 2014-05-25 02:20:13 +0000 + Commit: 8406092 + + SPARK-1911: Emphasize that Spark jars should be built with Java 6. + Patrick Wendell + 2014-05-24 18:27:00 -0700 + Commit: 217bd56, github.com/apache/spark/pull/859 + + [SPARK-1900 / 1918] PySpark on YARN is broken + Andrew Or + 2014-05-24 18:01:49 -0700 + Commit: 12f5ecc, github.com/apache/spark/pull/853 + + Update LBFGSSuite.scala + baishuo(白硕) + 2014-05-23 13:02:40 -0700 + Commit: 9be103a, github.com/apache/spark/pull/815 + + Updated scripts for auditing releases + Tathagata Das + 2014-05-22 20:48:55 -0700 + Commit: 6541ca2, github.com/apache/spark/pull/844 + + [SPARK-1896] Respect spark.master (and --master) before MASTER in spark-shell + Andrew Or + 2014-05-22 20:32:27 -0700 + Commit: c3b4065, github.com/apache/spark/pull/846 + + [SPARK-1897] Respect spark.jars (and --jars) in spark-shell + Andrew Or + 2014-05-22 20:25:41 -0700 + Commit: 23cc40e, github.com/apache/spark/pull/849 + + Fix UISuite unit test that fails under Jenkins contention + Aaron Davidson + 2014-05-22 15:11:05 -0700 + Commit: a566216, github.com/apache/spark/pull/857 + + [SPARK-1870] Make spark-submit --jars work in yarn-cluster mode. + Xiangrui Meng + 2014-05-22 01:52:50 -0700 + Commit: 79cd26c, github.com/apache/spark/pull/848 + + Configuration documentation updates + Reynold Xin + 2014-05-21 18:49:12 -0700 + Commit: 75af8bd, github.com/apache/spark/pull/851 + + [SPARK-1889] [SQL] Apply splitConjunctivePredicates to join condition while finding join ke... + Takuya UESHIN + 2014-05-21 15:37:47 -0700 + Commit: 6e7934e, github.com/apache/spark/pull/836 + + [SPARK-1519] Support minPartitions param of wholeTextFiles() in PySpark + Kan Zhang + 2014-05-21 13:26:53 -0700 + Commit: 30d1df5, github.com/apache/spark/pull/697 + + [Typo] Stoped -> Stopped + Andrew Or + 2014-05-21 11:59:05 -0700 + Commit: 9b8f772, github.com/apache/spark/pull/847 + + [Minor] Move JdbcRDDSuite to the correct package + Andrew Or + 2014-05-21 01:25:10 -0700 + Commit: bc6bbfa, github.com/apache/spark/pull/839 + + [Docs] Correct example of creating a new SparkConf + Andrew Or + 2014-05-21 01:23:34 -0700 + Commit: 7295dd9, github.com/apache/spark/pull/842 + + [SPARK-1250] Fixed misleading comments in bin/pyspark, bin/spark-class + Sumedh Mungee + 2014-05-21 01:22:25 -0700 + Commit: 364c14a, github.com/apache/spark/pull/843 + + [Hotfix] Blacklisted flaky HiveCompatibility test + Tathagata Das + 2014-05-20 10:27:12 -0700 + Commit: b4d93d3, github.com/apache/spark/pull/838 + + Updated CHANGES.txt + Tathagata Das + 2014-05-19 23:12:24 -0700 + Commit: 1c00f2a + + [Spark 1877] ClassNotFoundException when loading RDD with serialized objects + Tathagata Das , Ghidireac + 2014-05-19 22:36:24 -0700 + Commit: 6cbe2a3, github.com/apache/spark/pull/835 + + [SPARK-1874][MLLIB] Clean up MLlib sample data + Xiangrui Meng + 2014-05-19 21:29:33 -0700 + Commit: 1c6c8b5, github.com/apache/spark/pull/833 + + SPARK-1689: Spark application should die when removed by Master + Aaron Davidson + 2014-05-19 20:55:26 -0700 + Commit: 78b6e6f, github.com/apache/spark/pull/832 + + [SPARK-1875]NoClassDefFoundError: StringUtils when building with hadoop 1.x and hive + witgo + 2014-05-19 19:40:29 -0700 + Commit: 875c54f, github.com/apache/spark/pull/824 + + SPARK-1879. Increase MaxPermSize since some of our builds have many classes + Matei Zaharia + 2014-05-19 18:42:28 -0700 + Commit: 00563e1, github.com/apache/spark/pull/823 + + SPARK-1878: Fix the incorrect initialization order + zsxwing + 2014-05-19 16:41:31 -0700 + Commit: 901102c, github.com/apache/spark/pull/822 + + [SPARK-1876] Windows fixes to deal with latest distribution layout changes + Matei Zaharia + 2014-05-19 15:02:35 -0700 + Commit: 111c121, github.com/apache/spark/pull/819 + + [WIP][SPARK-1871][MLLIB] Improve MLlib guide for v1.0 + Xiangrui Meng + 2014-05-18 17:00:57 -0700 + Commit: ecab8a2, github.com/apache/spark/pull/816 + + SPARK-1873: Add README.md file when making distributions + Patrick Wendell + 2014-05-18 16:51:53 -0700 + Commit: 8e8b351, github.com/apache/spark/pull/818 + + Fix spark-submit path in spark-shell & pyspark + Neville Li + 2014-05-18 13:31:23 -0700 + Commit: e06e4b0, github.com/apache/spark/pull/812 + + Make deprecation warning less severe + Patrick Wendell + 2014-05-16 22:58:47 -0700 + Commit: 3b3d7c8, github.com/apache/spark/pull/810 + + [SPARK-1824] Remove from Python examples + Andrew Or + 2014-05-16 22:36:23 -0700 + Commit: 03b4242, github.com/apache/spark/pull/802 + + [SPARK-1808] Route bin/pyspark through Spark submit + Andrew Or + 2014-05-16 22:34:38 -0700 + Commit: 318739a, github.com/apache/spark/pull/799 + + Version bump of spark-ec2 scripts + Patrick Wendell + 2014-05-16 21:42:14 -0700 + Commit: 9cd12f3, github.com/apache/spark/pull/809 + + SPARK-1864 Look in spark conf instead of system properties when propagating configuration to executors. + Michael Armbrust + 2014-05-16 20:25:10 -0700 + Commit: a16a19f, github.com/apache/spark/pull/808 + + Tweaks to Mesos docs + Matei Zaharia + 2014-05-16 17:35:05 -0700 + Commit: 2ba6711, github.com/apache/spark/pull/806 + + [SQL] Implement between in hql + Michael Armbrust + 2014-05-16 11:47:00 -0700 + Commit: 386b31c, github.com/apache/spark/pull/804 + + bugfix: overflow of graphx Edge compare function + Zhen Peng + 2014-05-16 11:37:18 -0700 + Commit: ff47cdc, github.com/apache/spark/pull/769 + + SPARK-1862: Support for MapR in the Maven build. + Patrick Wendell + 2014-05-15 23:31:43 -0700 + Commit: eec4dd8, github.com/apache/spark/pull/803 + + [Spark-1461] Deferred Expression Evaluation (short-circuit evaluation) + Cheng Hao + 2014-05-15 22:12:34 -0700 + Commit: eac4ee8, github.com/apache/spark/pull/446 + + SPARK-1860: Do not cleanup application work/ directories by default + Aaron Davidson + 2014-05-15 21:37:58 -0700 + Commit: 5441471, github.com/apache/spark/pull/800 + + Typos in Spark + Huajian Mao + 2014-05-15 18:20:16 -0700 + Commit: a2742d8, github.com/apache/spark/pull/798 + + Fixes a misplaced comment. + Prashant Sharma + 2014-05-15 16:58:37 -0700 + Commit: 2e418f5, github.com/apache/spark/pull/788 + + [SQL] Fix tiny/small ints from HiveMetastore. + Michael Armbrust + 2014-05-15 16:50:42 -0700 + Commit: ffa9c49, github.com/apache/spark/pull/797 + + SPARK-1803 Replaced colon in filenames with a dash + Stevo Slavić , Stevo Slavic + 2014-05-15 16:44:14 -0700 + Commit: 22f261a, github.com/apache/spark/pull/739 + + SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro file... + Sandy Ryza + 2014-05-15 16:35:39 -0700 + Commit: 3587057, github.com/apache/spark/pull/795 + + [SPARK-1741][MLLIB] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans + Xiangrui Meng + 2014-05-15 11:59:59 -0700 + Commit: bc9a96e, github.com/apache/spark/pull/670 + + [SPARK-1819] [SQL] Fix GetField.nullable. + Takuya UESHIN + 2014-05-15 11:21:33 -0700 + Commit: f9eeddc, github.com/apache/spark/pull/757 + + [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ... + Takuya UESHIN + 2014-05-15 11:20:21 -0700 + Commit: 7515367, github.com/apache/spark/pull/790 + + SPARK-1846 Ignore logs directory in RAT checks + Andrew Ash + 2014-05-15 11:05:39 -0700 + Commit: aa5f989, github.com/apache/spark/pull/793 + + HOTFIX: Don't build Javadoc in Maven when creating releases. + Patrick Wendell + 2014-05-14 23:48:03 -0700 + Commit: 88f1da3 + + fix different versions of commons-lang dependency and apache/spark#746 addendum + witgo + 2014-05-14 22:26:26 -0700 + Commit: 31b853c, github.com/apache/spark/pull/754 + + Package docs + Prashant Sharma , Patrick Wendell + 2014-05-14 22:24:41 -0700 + Commit: c02d614, github.com/apache/spark/pull/785 + + Documentation: Encourage use of reduceByKey instead of groupByKey. + Patrick Wendell + 2014-05-14 22:24:04 -0700 + Commit: f2faa37, github.com/apache/spark/pull/784 + + Add language tabs and Python version to interactive part of quick-start + Matei Zaharia + 2014-05-14 21:45:20 -0700 + Commit: 976784b, github.com/apache/spark/pull/782 + + [SPARK-1840] SparkListenerBus prints out scary error message when terminated normally + Tathagata Das + 2014-05-14 21:13:41 -0700 + Commit: ba87123, github.com/apache/spark/pull/783 + + default task number misleading in several places + Chen Chao + 2014-05-14 18:20:20 -0700 + Commit: 9f0f2ec, github.com/apache/spark/pull/766 + + [SPARK-1826] fix the head notation of package object dsl + wangfei + 2014-05-14 17:59:11 -0700 + Commit: fdf9717, github.com/apache/spark/pull/765 + + [Typo] propertes -> properties + andrewor14 + 2014-05-14 17:54:53 -0700 + Commit: 5ca3096, github.com/apache/spark/pull/780 + + [SPARK-1696][MLLIB] use alpha in dense dspr + Xiangrui Meng + 2014-05-14 17:18:30 -0700 + Commit: d6f1a75, github.com/apache/spark/pull/778 + + [FIX] do not load defaults when testing SparkConf in pyspark + Xiangrui Meng + 2014-05-14 14:57:17 -0700 + Commit: 31faec7, github.com/apache/spark/pull/775 + + SPARK-1833 - Have an empty SparkContext constructor. + Patrick Wendell + 2014-05-14 12:53:30 -0700 + Commit: 8e13ab2, github.com/apache/spark/pull/774 + + SPARK-1829 Sub-second durations shouldn't round to "0 s" + Andrew Ash + 2014-05-14 12:01:14 -0700 + Commit: 530bdf7, github.com/apache/spark/pull/768 + + Fix: sbt test throw an java.lang.OutOfMemoryError: PermGen space + witgo + 2014-05-14 11:19:26 -0700 + Commit: 379f733, github.com/apache/spark/pull/773 + + Adding back hive support + Patrick Wendell + 2014-05-14 10:21:27 -0700 + Commit: e8ca397 + + [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler + Mark Hamstra + 2014-05-14 10:07:25 -0700 + Commit: 9ff9078, github.com/apache/spark/pull/622 + + SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies + Patrick Wendell + 2014-05-14 09:51:01 -0700 + Commit: 34f6fa9, github.com/apache/spark/pull/767 + + SPARK-1818 Freshen Mesos documentation + Andrew Ash + 2014-05-14 09:45:33 -0700 + Commit: fc6b652, github.com/apache/spark/pull/756 + + SPARK-1827. LICENSE and NOTICE files need a refresh to contain transitive dependency info + Sean Owen + 2014-05-14 09:38:33 -0700 + Commit: 7083282, github.com/apache/spark/pull/770 + + Fixed streaming examples docs to use run-example instead of spark-submit + Tathagata Das + 2014-05-14 04:17:32 -0700 + Commit: c7571d8, github.com/apache/spark/pull/722 + + [SPARK-1769] Executor loss causes NPE race condition + Andrew Or + 2014-05-14 00:54:33 -0700 + Commit: 69ec314, github.com/apache/spark/pull/762 + + Fix dep exclusion: avro-ipc, not avro, depends on netty. + Marcelo Vanzin + 2014-05-14 00:37:57 -0700 + Commit: b3d9878, github.com/apache/spark/pull/763 + + SPARK-1801. expose InterruptibleIterator and TaskKilledException in deve... + Koert Kuipers + 2014-05-14 00:10:12 -0700 + Commit: 7da80a3, github.com/apache/spark/pull/764 + + [SQL] Improve column pruning. + Michael Armbrust + 2014-05-13 23:27:22 -0700 + Commit: f66f766, github.com/apache/spark/pull/729 + + Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per partition" + Patrick Wendell + 2014-05-13 23:25:19 -0700 + Commit: 721194b + + Implement ApproximateCountDistinct for SparkSql + larvaboy + 2014-05-13 21:26:08 -0700 + Commit: 92b0ec9, github.com/apache/spark/pull/737 + + [SPARK-1784] Add a new partitioner to allow specifying # of keys per partition + Syed Hashmi + 2014-05-13 21:24:23 -0700 + Commit: 66fe479, github.com/apache/spark/pull/721 + + [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. + Michael Armbrust + 2014-05-13 21:23:51 -0700 + Commit: 618b3e6, github.com/apache/spark/pull/761 + + [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePath + Ye Xianjin + 2014-05-13 19:03:51 -0700 + Commit: ef5e9d7, github.com/apache/spark/pull/436 + + [SPARK-1816] LiveListenerBus dies if a listener throws an exception + Andrew Or + 2014-05-13 18:32:32 -0700 + Commit: 3892ec5, github.com/apache/spark/pull/759 + + SPARK-1791 - SVM implementation does not use threshold parameter + Andrew Tulloch + 2014-05-13 17:31:27 -0700 + Commit: d6994f4, github.com/apache/spark/pull/725 + + BUILD: Add more content to make-distribution.sh. + Patrick Wendell + 2014-05-12 23:02:54 -0700 + Commit: 716462c + + Adding CHANGES.txt file and removing YARN support for now + Patrick Wendell + 2014-05-12 20:21:23 -0700 + Commit: fa2d4d8 + + SPARK-1815. SparkContext should not be marked DeveloperApi + Sandy Ryza + 2014-05-12 20:08:30 -0700 + Commit: 31d54c0, github.com/apache/spark/pull/753 + + [SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. + Andrew Or + 2014-05-12 19:44:14 -0700 + Commit: b9e41f4, github.com/apache/spark/pull/701 + + [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking around + Andrew Or + 2014-05-12 19:42:35 -0700 + Commit: 5ef24a0, github.com/apache/spark/pull/751 + + SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile() + Andrew Ash , Michael Armbrust + 2014-05-12 19:23:39 -0700 + Commit: b52ac0e, github.com/apache/spark/pull/690 + + Modify a typo in monitoring.md + Kousuke Saruta + 2014-05-12 19:21:06 -0700 + Commit: 89b56d7, github.com/apache/spark/pull/698 + + L-BFGS Documentation + DB Tsai + 2014-05-12 19:20:24 -0700 + Commit: bad4c9d, github.com/apache/spark/pull/702 + + Typo: resond -> respond + Andrew Ash + 2014-05-12 18:46:28 -0700 + Commit: 1fbebca, github.com/apache/spark/pull/743 + + [SQL] Make Hive Metastore conversion functions publicly visible. + Michael Armbrust + 2014-05-12 18:40:30 -0700 + Commit: 24cc933, github.com/apache/spark/pull/750 + + [SPARK-1736] Spark submit for Windows + Andrew Or + 2014-05-12 17:39:40 -0700 + Commit: 59695b3, github.com/apache/spark/pull/745 + + SPARK-1802. (Addendium) Audit dependency graph when Spark is built with -Pyarn + Sean Owen + 2014-05-12 17:35:29 -0700 + Commit: 02caa7e, github.com/apache/spark/pull/746 + + SPARK-1623: Use File objects instead of String's in HTTPBroadcast + Patrick Wendell + 2014-05-12 17:27:28 -0700 + Commit: c294f37, github.com/apache/spark/pull/749 + + Rename testExecutorEnvs --> executorEnvs. + Patrick Wendell + 2014-05-12 17:09:13 -0700 + Commit: e9d602d, github.com/apache/spark/pull/747 + + Adding hadoop-2.2 profile to the build + Patrick Wendell + 2014-05-12 15:40:48 -0700 + Commit: b66051e + + Rollback versions for 1.0.0-rc4 + Patrick Wendell + 2014-05-12 15:23:53 -0700 + Commit: 51142b7 + + SPARK-1802. Audit dependency graph when Spark is built with -Phive + Sean Owen + 2014-05-12 14:17:25 -0700 + Commit: e185281, github.com/apache/spark/pull/744 + + SPARK-1798. Tests should clean up temp files + Sean Owen + 2014-05-12 14:16:19 -0700 + Commit: 14515b4, github.com/apache/spark/pull/732 + + BUILD: Include Hive with default packages when creating a release + Patrick Wendell + 2014-05-12 13:20:23 -0700 + Commit: 722ecaa + + SPARK-1786: Reopening PR 724 + Ankur Dave , Joseph E. Gonzalez + 2014-05-12 13:05:24 -0700 + Commit: 642ad49, github.com/apache/spark/pull/742 + + SPARK-1806: Upgrade Mesos dependency to 0.18.1 + Bernardo Gomez Palacio + 2014-05-12 11:10:28 -0700 + Commit: 0be8b45, github.com/apache/spark/pull/741 + + SPARK-1772 Stop catching Throwable, let Executors die + Aaron Davidson + 2014-05-12 11:08:52 -0700 + Commit: c88adbb, github.com/apache/spark/pull/715 + + Revert "SPARK-1786: Edge Partition Serialization" + Patrick Wendell + 2014-05-12 10:51:01 -0700 + Commit: 19ccf20 + + SPARK-1786: Edge Partition Serialization + Ankur Dave , Joseph E. Gonzalez + 2014-05-11 19:20:42 -0700 + Commit: 09e7aa4, github.com/apache/spark/pull/724 + + Fix error in 2d Graph Partitioner + Joseph E. Gonzalez + 2014-05-11 18:33:46 -0700 + Commit: f84b798, github.com/apache/spark/pull/709 + + SPARK-1652: Set driver memory correctly in spark-submit. + Patrick Wendell + 2014-05-11 18:17:34 -0700 + Commit: 2eea663, github.com/apache/spark/pull/730 + + SPARK-1770: Load balance elements when repartitioning. + Patrick Wendell + 2014-05-11 17:11:55 -0700 + Commit: 5d69699, github.com/apache/spark/pull/727 + + remove outdated runtime Information scala home + witgo + 2014-05-11 14:34:27 -0700 + Commit: 8097bb2, github.com/apache/spark/pull/728 + + Revert "Enabled incremental build that comes with sbt 0.13.2" + Patrick Wendell + 2014-05-10 21:08:53 -0700 + Commit: 758e543 + + Enabled incremental build that comes with sbt 0.13.2 + Prashant Sharma + 2014-05-10 21:08:04 -0700 + Commit: 71ce7eb, github.com/apache/spark/pull/525 + + [SPARK-1774] Respect SparkSubmit --jars on YARN (client) + Andrew Or + 2014-05-10 20:58:02 -0700 + Commit: 012f904, github.com/apache/spark/pull/710 + + SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite failure + Sean Owen + 2014-05-10 20:50:40 -0700 + Commit: c7253da, github.com/apache/spark/pull/723 + + Unify GraphImpl RDDs + other graph load optimizations + Ankur Dave + 2014-05-10 14:48:07 -0700 + Commit: 4e9a0cb, github.com/apache/spark/pull/497 + + [SPARK-1690] Tolerating empty elements when saving Python RDD to text files + Kan Zhang + 2014-05-10 14:01:08 -0700 + Commit: ac86af8, github.com/apache/spark/pull/644 + + Add Python includes to path before depickling broadcast values + Bouke van der Bijl + 2014-05-10 13:02:13 -0700 + Commit: 2a669a7, github.com/apache/spark/pull/656 + + fix broken in link in python docs + Andy Konwinski + 2014-05-10 12:46:51 -0700 + Commit: 71ad53f, github.com/apache/spark/pull/650 + + SPARK-1708. Add a ClassTag on Serializer and things that depend on it + Matei Zaharia + 2014-05-10 12:10:24 -0700 + Commit: 9fbb22c, github.com/apache/spark/pull/700 + + [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD. + Takuya UESHIN + 2014-05-10 12:03:27 -0700 + Commit: 7486474, github.com/apache/spark/pull/711 + + [SQL] Upgrade parquet library. + Michael Armbrust + 2014-05-10 11:48:01 -0700 + Commit: a61b71c, github.com/apache/spark/pull/684 + + [SPARK-1644] The org.datanucleus:* should not be packaged into spark-assembly-*.jar + witgo + 2014-05-10 10:15:04 -0700 + Commit: 2a878da, github.com/apache/spark/pull/688 + + SPARK-1686: keep schedule() calling in the main thread + CodingCat + 2014-05-09 21:50:23 -0700 + Commit: adf8cdd, github.com/apache/spark/pull/639 + + SPARK-1770: Revert accidental(?) fix + Aaron Davidson + 2014-05-09 14:51:34 -0700 + Commit: 8202276, github.com/apache/spark/pull/716 + + [SPARK-1760]: fix building spark with maven documentation + witgo + 2014-05-09 01:51:26 -0700 + Commit: 80f292a, github.com/apache/spark/pull/712 + + Converted bang to ask to avoid scary warning when a block is removed + Tathagata Das + 2014-05-08 22:34:08 -0700 + Commit: b8c17e3, github.com/apache/spark/pull/708 + + MINOR: Removing dead code. + Patrick Wendell + 2014-05-08 22:33:06 -0700 + Commit: 1d56cd5 + + SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfo + Sandeep + 2014-05-08 22:30:17 -0700 + Commit: 5c8e8de, github.com/apache/spark/pull/707 + + SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`. + Patrick Wendell + 2014-05-08 22:26:17 -0700 + Commit: f6323eb, github.com/apache/spark/pull/704 + + [SPARK-1631] Correctly set the Yarn app name when launching the AM. + Marcelo Vanzin + 2014-05-08 20:46:11 -0700 + Commit: 7e19334, github.com/apache/spark/pull/539 + + [SPARK-1755] Respect SparkSubmit --name on YARN + Andrew Or + 2014-05-08 20:45:29 -0700 + Commit: 666bebe, github.com/apache/spark/pull/699 + + Include the sbin/spark-config.sh in spark-executor + Bouke van der Bijl + 2014-05-08 20:43:37 -0700 + Commit: ab91227, github.com/apache/spark/pull/651 + + Bug fix of sparse vector conversion + Funes , funes + 2014-05-08 17:54:10 -0700 + Commit: 9ed17ff, github.com/apache/spark/pull/661 + + [SPARK-1157][MLlib] Bug fix: lossHistory should exclude rejection steps, and remove miniBatch + DB Tsai + 2014-05-08 17:53:22 -0700 + Commit: 3452997, github.com/apache/spark/pull/582 + + MLlib documentation fix + DB Tsai + 2014-05-08 17:52:32 -0700 + Commit: d81d626, github.com/apache/spark/pull/703 + + Fixing typo in als.py + Evan Sparks + 2014-05-08 13:07:30 -0700 + Commit: 98944a9, github.com/apache/spark/pull/696 + + [SPARK-1754] [SQL] Add missing arithmetic DSL operations. + Takuya UESHIN + 2014-05-08 15:31:47 -0700 + Commit: 6f701ff, github.com/apache/spark/pull/689 + + [SPARK-1745] Move interrupted flag from TaskContext constructor (minor) + Andrew Or + 2014-05-08 12:13:07 -0700 + Commit: ee63321, github.com/apache/spark/pull/675 + + SPARK-1565, update examples to be used with spark-submit script. + Prashant Sharma + 2014-05-08 10:23:05 -0700 + Commit: 30cfa8d, github.com/apache/spark/pull/552 + + [SQL] Improve SparkSQL Aggregates + Michael Armbrust + 2014-05-08 01:08:43 -0400 + Commit: 8f3b925, github.com/apache/spark/pull/683 + + Use numpy directly for matrix multiply. + Evan Sparks + 2014-05-08 00:24:36 -0400 + Commit: 010040f, github.com/apache/spark/pull/687 + + SPARK-1668: Add implicit preference as an option to examples/MovieLensALS + Sandeep + 2014-05-08 00:15:05 -0400 + Commit: 35aa244, github.com/apache/spark/pull/597 + + SPARK-1544 Add support for deep decision trees. + Manish Amde , manishamde , Evan Sparks + 2014-05-07 17:08:38 -0700 + Commit: c7b2704, github.com/apache/spark/pull/475 + + Update GradientDescentSuite.scala + baishuo(白硕) + 2014-05-07 16:02:55 -0700 + Commit: 0972b62, github.com/apache/spark/pull/588 + + [SPARK-1743][MLLIB] add loadLibSVMFile and saveAsLibSVMFile to pyspark + Xiangrui Meng + 2014-05-07 16:01:11 -0700 + Commit: bb90e87, github.com/apache/spark/pull/672 + + SPARK-1569 Spark on Yarn, authentication broken by pr299 + Thomas Graves + 2014-05-07 15:51:53 -0700 + Commit: 879eeee, github.com/apache/spark/pull/649 + + [SPARK-1688] Propagate PySpark worker stderr to driver + Andrew Or + 2014-05-07 14:35:22 -0700 + Commit: 82c8e89, github.com/apache/spark/pull/603 + + Typo fix: fetchting -> fetching + Andrew Ash + 2014-05-07 17:24:49 -0400 + Commit: 0759ee7, github.com/apache/spark/pull/680 + + Nicer logging for SecurityManager startup + Andrew Ash + 2014-05-07 17:24:12 -0400 + Commit: 69e2726, github.com/apache/spark/pull/678 + + [SQL] Fix Performance Issue in data type casting + Cheng Hao + 2014-05-07 16:54:58 -0400 + Commit: 82ceda2, github.com/apache/spark/pull/679 + + SPARK-1579: Clean up PythonRDD and avoid swallowing IOExceptions + Aaron Davidson + 2014-05-07 09:48:31 -0700 + Commit: 18caa8c, github.com/apache/spark/pull/640 + + [SPARK-1460] Returning SchemaRDD instead of normal RDD on Set operations... + Kan Zhang + 2014-05-07 09:41:31 -0700 + Commit: da9f9e0, github.com/apache/spark/pull/448 + + [WIP][Spark-SQL] Optimize the Constant Folding for Expression + Cheng Hao , Michael Armbrust + 2014-05-07 03:37:12 -0400 + Commit: 756c969, github.com/apache/spark/pull/482 + + SPARK-1746: Support setting SPARK_JAVA_OPTS on executors for backwards compatibility + Patrick Wendell + 2014-05-07 00:11:05 -0700 + Commit: 00fac73, github.com/apache/spark/pull/676 + + [HOTFIX] SPARK-1637: There are some Streaming examples added after the PR #571 was last updated. + Sandeep + 2014-05-06 21:55:05 -0700 + Commit: ade4756, github.com/apache/spark/pull/673 + + Proposal: clarify Scala programming guide on caching ... + Ethan Jewett + 2014-05-06 20:50:08 -0700 + Commit: 51e2775, github.com/apache/spark/pull/668 + + SPARK-1727. Correct small compile errors, typos, and markdown issues in (primarly) MLlib docs + Sean Owen + 2014-05-06 20:07:22 -0700 + Commit: 514ee93, github.com/apache/spark/pull/653 + + SPARK-1637: Clean up examples for 1.0 + Sandeep + 2014-05-06 17:27:52 -0700 + Commit: 8cfebf5, github.com/apache/spark/pull/571 + + SPARK-1737: Warn rather than fail when Java 7+ is used to create distributions + Patrick Wendell + 2014-05-06 15:41:46 -0700 + Commit: d8f1b33, github.com/apache/spark/pull/669 + + [SPARK-1549] Add Python support to spark-submit + Matei Zaharia + 2014-05-06 15:12:35 -0700 + Commit: d7ddb26, github.com/apache/spark/pull/664 + + SPARK-1734: spark-submit throws an exception: Exception in thread "main"... + witgo + 2014-05-06 14:17:39 -0700 + Commit: 48cc9a9, github.com/apache/spark/pull/665 + + [SPARK-1685] Cancel retryTimer on restart of Worker or AppClient + Mark Hamstra + 2014-05-06 12:53:39 -0700 + Commit: 0aaa2c6, github.com/apache/spark/pull/602 + + Fix two download suggestions in the docs: + Patrick Wendell + 2014-05-06 12:07:46 -0700 + Commit: 1083f2b, github.com/apache/spark/pull/662 + + SPARK-1474: Spark on yarn assembly doesn't include AmIpFilter + Thomas Graves + 2014-05-06 12:00:09 -0700 + Commit: 0c3e415, github.com/apache/spark/pull/406 + + Update OpenHashSet.scala + ArcherShao + 2014-05-06 10:12:59 -0700 + Commit: 4ff3929, github.com/apache/spark/pull/667 + + [SQL] SPARK-1732 - Support for null primitive values. + Michael Armbrust + 2014-05-05 22:59:42 -0700 + Commit: 39ac62d, github.com/apache/spark/pull/658 + + [SPARK-1735] Add the missing special profiles to make-distribution.sh + Andrew Or + 2014-05-05 22:14:47 -0700 + Commit: 4708eff, github.com/apache/spark/pull/660 + + [SPARK-1678][SPARK-1679] In-memory compression bug fix and made compression configurable, disabled by default + Cheng Lian + 2014-05-05 19:38:59 -0700 + Commit: 2853e56, github.com/apache/spark/pull/608 + + [SPARK-1594][MLLIB] Cleaning up MLlib APIs and guide + Xiangrui Meng + 2014-05-05 18:32:54 -0700 + Commit: 32c960a, github.com/apache/spark/pull/524 + + Expose SparkListeners and relevant classes as DeveloperApi + Andrew Or + 2014-05-05 18:32:14 -0700 + Commit: a5f765c, github.com/apache/spark/pull/648 + + SPARK-1728. JavaRDDLike.mapPartitionsWithIndex requires ClassTag + Sandy Ryza + 2014-05-05 18:26:34 -0700 + Commit: 01e3ff0, github.com/apache/spark/pull/657 + + [SPARK-1681] Include datanucleus jars in Spark Hive distribution + Andrew Or + 2014-05-05 16:28:07 -0700 + Commit: 4d0dd50, github.com/apache/spark/pull/610 + + [SPARK-1504], [SPARK-1505], [SPARK-1558] Updated Spark Streaming guide + Tathagata Das + 2014-05-05 15:28:19 -0700 + Commit: 1fac4ec, github.com/apache/spark/pull/652 + + SPARK-1721: Reset the thread classLoader in the Mesos Executor + Bouke van der Bijl + 2014-05-05 11:19:35 -0700 + Commit: 80f4360, github.com/apache/spark/pull/620 + + SPARK-1556. jets3t dep doesn't update properly with newer Hadoop versions + Sean Owen + 2014-05-05 10:33:49 -0700 + Commit: 5d72283, github.com/apache/spark/pull/629 + + Updated doc for spark.closure.serializer to indicate only Java serializer work. + Reynold Xin + 2014-05-05 00:52:06 -0700 + Commit: 6be7226, github.com/apache/spark/pull/642 + + Address SPARK-1717 + msiddalingaiah + 2014-05-04 21:59:10 -0700 + Commit: b5c62c8, github.com/apache/spark/pull/641 + + SPARK-1710: spark-submit should print better errors than "InvocationTargetException" + Sandeep + 2014-05-04 20:51:53 -0700 + Commit: afcb9ae, github.com/apache/spark/pull/630 + + EC2 script should exit with non-zero code on UsageError + Allan Douglas R. de Oliveira + 2014-05-04 20:36:51 -0700 + Commit: 7305278, github.com/apache/spark/pull/638 + + SPARK-1693: Most of the tests throw a java.lang.SecurityException when s... + witgo + 2014-05-04 17:48:52 -0700 + Commit: ec0bce1, github.com/apache/spark/pull/628 + + SPARK-1629. Addendum: Depend on commons lang3 (already used by tachyon) as it's used in ReplSuite, and return to use lang3 utility in Utils.scala + Sean Owen + 2014-05-04 17:43:28 -0700 + Commit: 4505bc2, github.com/apache/spark/pull/635 + + SPARK-1703 Warn users if Spark is run on JRE6 but compiled with JDK7. + Patrick Wendell + 2014-05-04 12:22:23 -0700 + Commit: 2f091d5, github.com/apache/spark/pull/627 + + SPARK-1663. (Addendum) Fix signature of one version of JavaPairRDDStream.reduceByKeyAndWindow() + Sean Owen + 2014-05-04 11:55:29 -0700 + Commit: 40d05a4, github.com/apache/spark/pull/633 + + SPARK-1658: Correctly identify if maven is installed and working + Rahul Singhal + 2014-05-04 11:08:39 -0700 + Commit: 2ee5f04, github.com/apache/spark/pull/580 + + The default version of yarn is equal to the hadoop version + witgo + 2014-05-03 23:32:12 -0700 + Commit: acbf307, github.com/apache/spark/pull/626 + + Whitelist Hive Tests + Michael Armbrust + 2014-05-03 23:13:51 -0700 + Commit: e88a636, github.com/apache/spark/pull/596 + + [SQL] Better logging when applying rules. + Michael Armbrust + 2014-05-03 18:38:44 -0700 + Commit: e24d5cd, github.com/apache/spark/pull/616 + + EC2 configurable workers + Allan Douglas R. de Oliveira + 2014-05-03 16:52:19 -0700 + Commit: 8406ac4, github.com/apache/spark/pull/612 + + SPARK-1689 AppClient should indicate app is dead() when removed + Aaron Davidson + 2014-05-03 13:27:10 -0700 + Commit: 36e687d, github.com/apache/spark/pull/605 + + [Bugfix] Tachyon file cleanup logical error + Cheng Lian + 2014-05-03 13:23:52 -0700 + Commit: bc3bfea, github.com/apache/spark/pull/575 + + SPARK-1663. Corrections for several compile errors in streaming code examples, and updates to follow API changes + Sean Owen + 2014-05-03 12:31:31 -0700 + Commit: 08c4d11, github.com/apache/spark/pull/589 + + [WIP] SPARK-1676: Cache Hadoop UGIs by default to prevent FileSystem leak + Thomas Graves + 2014-05-03 10:59:05 -0700 + Commit: 0441515, github.com/apache/spark/pull/621 + + Update SchemaRDD.scala + ArcherShao + 2014-05-03 00:17:36 -0700 + Commit: 34f22bc, github.com/apache/spark/pull/619 + + SPARK-1700: Close socket file descriptors on task completion + Aaron Davidson + 2014-05-02 23:55:13 -0700 + Commit: d2cbd3d, github.com/apache/spark/pull/623 + + SPARK-1492. Update Spark YARN docs to use spark-submit + Sandy Ryza + 2014-05-02 21:42:31 -0700 + Commit: a314342, github.com/apache/spark/pull/601 + + delete no use var + wangfei + 2014-05-02 21:34:54 -0700 + Commit: b65def7, github.com/apache/spark/pull/613 + + SPARK-1695: java8-tests compiler error: package com.google.common.co... + witgo + 2014-05-02 12:40:27 -0700 + Commit: d28c058, github.com/apache/spark/pull/611 + + Add tests for FileLogger, EventLoggingListener, and ReplayListenerBus + Andrew Or + 2014-05-01 21:42:06 -0700 + Commit: d4c8af8, github.com/apache/spark/pull/591 + + SPARK-1659: improvements spark-submit usage + witgo + 2014-05-01 21:39:40 -0700 + Commit: 18595dd, github.com/apache/spark/pull/581 + + fix the spelling mistake + wangfei + 2014-05-01 21:37:22 -0700 + Commit: 35ca6c5, github.com/apache/spark/pull/614 + + [SQL] SPARK-1661 - Fix regex_serde test + Michael Armbrust + 2014-05-01 21:32:43 -0700 + Commit: d339b33, github.com/apache/spark/pull/595 + + SPARK-1691: Support quoted arguments inside of spark-submit. + Patrick Wendell + 2014-05-01 01:15:51 -0700 + Commit: dd601bf, github.com/apache/spark/pull/609 + + Fix SPARK-1629: Spark should inline use of commons-lang `SystemUtils.IS_... + witgo + 2014-04-30 09:49:45 -0700 + Commit: 74bb88b, github.com/apache/spark/pull/569 + + SPARK-1004. PySpark on YARN + Sandy Ryza + 2014-04-29 23:24:34 -0700 + Commit: 177361c, github.com/apache/spark/pull/30 + + Handle the vals that never used + WangTao + 2014-04-29 22:07:20 -0700 + Commit: b0ded1f, github.com/apache/spark/pull/565 + + Args for worker rather than master + Chen Chao + 2014-04-29 22:05:40 -0700 + Commit: 775020f, github.com/apache/spark/pull/587 + + [SPARK-1646] Micro-optimisation of ALS + Tor Myklebust + 2014-04-29 22:04:34 -0700 + Commit: 92269f9, github.com/apache/spark/pull/568 + + [SPARK-1674] fix interrupted system call error in pyspark's RDD.pipe + Xiangrui Meng + 2014-04-29 18:06:45 -0700 + Commit: 919ed31, github.com/apache/spark/pull/594 + + SPARK-1588. Restore SPARK_YARN_USER_ENV and SPARK_JAVA_OPTS for YARN. + Sandy Ryza + 2014-04-29 12:54:02 -0700 + Commit: 5f48721, github.com/apache/spark/pull/586 + + SPARK-1509: add zipWithIndex zipWithUniqueId methods to java api + witgo + 2014-04-29 11:30:47 -0700 + Commit: 9754d1b, github.com/apache/spark/pull/423 + + SPARK-1557 Set permissions on event log files/directories + Thomas Graves + 2014-04-29 09:19:48 -0500 + Commit: bccd13e, github.com/apache/spark/pull/538 + + HOTFIX: minor change to release script + Patrick Wendell + 2014-04-29 00:59:38 -0700 + Commit: c27ce2b + + HOTFIX: minor change to release script + Patrick Wendell + 2014-04-29 00:53:32 -0700 + Commit: 838cb0e + + [SPARK-1636][MLLIB] Move main methods to examples + Xiangrui Meng + 2014-04-29 00:41:03 -0700 + Commit: aa519e3, github.com/apache/spark/pull/584 + + Minor fix to python table caching API. + Michael Armbrust + 2014-04-29 00:36:15 -0700 + Commit: 0995787, github.com/apache/spark/pull/585 + + HOTFIX: Bug in release script + Patrick Wendell + 2014-04-29 00:10:17 -0700 + Commit: 4ed58aa + + Manual revert of rc2 version changes. + Patrick Wendell + 2014-04-28 22:59:09 -0700 + Commit: 2863344 + + Improved build configuration + witgo + 2014-04-28 22:50:51 -0700 + Commit: ee96460, github.com/apache/spark/pull/480 + + SPARK-1652: Remove incorrect deprecation warning in spark-submit + Patrick Wendell + 2014-04-28 18:14:59 -0700 + Commit: 42ba706, github.com/apache/spark/pull/578 + + SPARK-1654 and SPARK-1653: Fixes in spark-submit. + Patrick Wendell + 2014-04-28 17:29:22 -0700 + Commit: 2c9ce20, github.com/apache/spark/pull/576 + + SPARK-1652: Spark submit should fail gracefully if YARN not enabled + Patrick Wendell + 2014-04-28 17:26:57 -0700 + Commit: 38bf23e, github.com/apache/spark/pull/579 + + Changes to dev release script + Patrick Wendell + 2014-04-28 13:58:42 -0700 + Commit: 32d9db3 + + [SPARK-1633][Streaming] Java API unit test and example for custom streaming receiver in Java + Tathagata Das + 2014-04-28 13:58:09 -0700 + Commit: 6d89faf, github.com/apache/spark/pull/558 + + [SQL]Append some missing types for HiveUDF + Cheng Hao + 2014-04-27 23:59:42 -0700 + Commit: 42cb3b4, github.com/apache/spark/pull/459 + + Update the import package name for TestHive in sbt shell + Cheng Hao + 2014-04-27 23:57:29 -0700 + Commit: eb9308e, github.com/apache/spark/pull/574 + + Fix SPARK-1609: Executor fails to start when Command.extraJavaOptions contains multiple Java options + witgo + 2014-04-27 19:41:02 -0700 + Commit: 7bbf313, github.com/apache/spark/pull/547 + + SPARK-1145: Memory mapping with many small blocks can cause JVM allocation failures + Patrick Wendell + 2014-04-27 17:40:56 -0700 + Commit: 2f24159, github.com/apache/spark/pull/43 + + HOTFIX: Minor patch to merge script. + Patrick Wendell + 2014-04-27 15:45:17 -0700 + Commit: 99285d0 + + SPARK-1651: Delete existing deployment directory + Rahul Singhal + 2014-04-27 15:50:48 -0700 + Commit: 3c6c6c2, github.com/apache/spark/pull/573 + + SPARK-1648 Support closing JIRA's as part of merge script. + Patrick Wendell + 2014-04-27 15:41:57 -0700 + Commit: da26f9b, github.com/apache/spark/pull/570 + + SPARK-1650: Correctly identify maven project version + Rahul Singhal + 2014-04-27 15:17:06 -0700 + Commit: 98b13e0, github.com/apache/spark/pull/572 + + SPARK-1606: Infer user application arguments instead of requiring --arg. + Patrick Wendell + 2014-04-26 19:24:29 -0700 + Commit: ce57624, github.com/apache/spark/pull/563 + + SPARK-1467: Make StorageLevel.apply() factory methods Developer APIs + Sandeep + 2014-04-26 19:04:33 -0700 + Commit: 18ecc63, github.com/apache/spark/pull/551 + + [SPARK-1608] [SQL] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. + Takuya UESHIN + 2014-04-26 14:39:54 -0700 + Commit: dcea67f, github.com/apache/spark/pull/532 + + add note of how to support table with more than 22 fields + wangfei + 2014-04-26 14:38:42 -0700 + Commit: a020686, github.com/apache/spark/pull/564 + + [Spark-1382] Fix NPE in DStream.slice (updated version of #365) + zsxwing , Tathagata Das + 2014-04-25 19:04:34 -0700 + Commit: f85c681, github.com/apache/spark/pull/562 + + SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyM... + Sandy Ryza + 2014-04-25 17:55:04 -0700 + Commit: 94c71e0, github.com/apache/spark/pull/559 + + SPARK-1235: manage the DAGScheduler EventProcessActor with supervisor and refactor the DAGScheduler with Akka + CodingCat , Xiangrui Meng , Nan Zhu + 2014-04-25 16:04:48 -0700 + Commit: 5673c1e, github.com/apache/spark/pull/186 + + SPARK-1607. HOTFIX: Fix syntax adapting Int result to Short + Sean Owen + 2014-04-25 14:17:38 -0700 + Commit: 784b2a6, github.com/apache/spark/pull/556 + + Update KafkaWordCount.scala + baishuo(白硕) + 2014-04-25 13:18:49 -0700 + Commit: a0912a8, github.com/apache/spark/pull/523 + + Delete the val that never used + WangTao + 2014-04-25 11:47:01 -0700 + Commit: 97bfeda, github.com/apache/spark/pull/553 + + SPARK-1621 Upgrade Chill to 0.3.6 + Matei Zaharia + 2014-04-25 11:12:41 -0700 + Commit: 2c8dfd4, github.com/apache/spark/pull/543 + + SPARK-1619 Launch spark-shell with spark-submit + Patrick Wendell + 2014-04-24 23:59:16 -0700 + Commit: 8ba7f40, github.com/apache/spark/pull/542 + + SPARK-1607. Replace octal literals, removed in Scala 2.11, with hex literals + Sean Owen + 2014-04-24 23:34:00 -0700 + Commit: 7493ca9, github.com/apache/spark/pull/529 + + Call correct stop(). + Aaron Davidson + 2014-04-24 23:22:03 -0700 + Commit: 3eba9bd, github.com/apache/spark/pull/527 + + SPARK-1242 Add aggregate to python rdd + Holden Karau + 2014-04-24 23:07:54 -0700 + Commit: f09a2c0, github.com/apache/spark/pull/139 + + Fix [SPARK-1078]: Remove the Unnecessary lift-json dependency + Sandeep + 2014-04-24 21:51:52 -0700 + Commit: 496b9ae, github.com/apache/spark/pull/536 + + [Typo] In the maven docs: chd -> cdh + Andrew Or + 2014-04-24 21:51:17 -0700 + Commit: db69841, github.com/apache/spark/pull/548 + + Generalize pattern for planning hash joins. + Michael Armbrust + 2014-04-24 21:42:33 -0700 + Commit: ab131ab, github.com/apache/spark/pull/418 + + [SPARK-1617] and [SPARK-1618] Improvements to streaming ui and bug fix to socket receiver + Tathagata Das + 2014-04-24 21:34:37 -0700 + Commit: d933c71, github.com/apache/spark/pull/540 + + SPARK-1584: Upgrade Flume dependency to 1.4.0 + tmalaska + 2014-04-24 20:31:17 -0700 + Commit: 777a9a5, github.com/apache/spark/pull/507 + + SPARK-1586 Windows build fixes + Mridul Muralidharan + 2014-04-24 20:48:33 -0700 + Commit: 51a387a, github.com/apache/spark/pull/505 + + [SPARK-986]: Job cancelation for PySpark + Ahir Reddy + 2014-04-24 20:21:10 -0700 + Commit: 7b6d774, github.com/apache/spark/pull/541 + + [SPARK-1615] Synchronize accesses to the LiveListenerBus' event queue + Andrew Or + 2014-04-24 20:18:15 -0700 + Commit: 963046c, github.com/apache/spark/pull/544 + + [SPARK-1510] Spark Streaming metrics source for metrics system + jerryshao , Tathagata Das + 2014-04-24 18:56:57 -0700 + Commit: 0bc0f36, github.com/apache/spark/pull/545 + + Spark 1489 Fix the HistoryServer view acls + Thomas Graves + 2014-04-24 18:38:10 -0700 + Commit: c8dd132, github.com/apache/spark/pull/509 + + [SQL] Add support for parsing indexing into arrays in SQL. + Michael Armbrust + 2014-04-24 18:21:00 -0700 + Commit: 2a35fba, github.com/apache/spark/pull/518 + + [SPARK-1592][streaming] Automatically remove streaming input blocks + Tathagata Das + 2014-04-24 18:18:22 -0700 + Commit: a3b6d85, github.com/apache/spark/pull/512 + + SPARK-1438 RDD.sample() make seed param optional + Arun Ramakrishnan + 2014-04-24 17:27:16 -0700 + Commit: 521d435, github.com/apache/spark/pull/477 + + SPARK-1104: kill Process in workerThread of ExecutorRunner + CodingCat + 2014-04-24 15:55:18 -0700 + Commit: a1f8779, github.com/apache/spark/pull/35 + + Fix Scala Style + Sandeep + 2014-04-24 15:07:23 -0700 + Commit: 2250c7a, github.com/apache/spark/pull/531 + + SPARK-1494 Don't initialize classes loaded by MIMA excludes, attempt 2 + Michael Armbrust + 2014-04-24 14:54:01 -0700 + Commit: 5ca01f6, github.com/apache/spark/pull/526 + + Spark 1490 Add kerberos support to the HistoryServer + Thomas Graves + 2014-04-24 11:15:12 -0700 + Commit: 866b03e, github.com/apache/spark/pull/513 + + SPARK-1611: Fix incorrect initialization order in AppendOnlyMap + zsxwing + 2014-04-24 11:13:40 -0700 + Commit: 00a3ccc, github.com/apache/spark/pull/534 + + SPARK-1488. Squash more language feature warnings in new commits by importing implicitConversion + Sean Owen + 2014-04-24 10:06:18 -0700 + Commit: 8d92d93, github.com/apache/spark/pull/528 + + Small changes to release script + Patrick Wendell + 2014-04-24 09:59:44 -0700 + Commit: 563be2f + + [SPARK-1610] [SQL] Fix Cast to use exact type value when cast from BooleanType to NumericTy... + Takuya UESHIN + 2014-04-24 09:57:28 -0700 + Commit: 8f8e051, github.com/apache/spark/pull/533 + + SPARK-1601 & SPARK-1602: two bug fixes related to cancellation + Reynold Xin + 2014-04-24 00:27:45 -0700 + Commit: f98aac9, github.com/apache/spark/pull/521 + + SPARK-1587 Fix thread leak + Mridul Muralidharan + 2014-04-23 23:20:55 -0700 + Commit: 8684a15, github.com/apache/spark/pull/504 + + [Fix #79] Replace Breakable For Loops By While Loops + Sandeep + 2014-04-23 22:47:59 -0700 + Commit: e890771, github.com/apache/spark/pull/503 + + SPARK-1589: Fix the incorrect compare + zsxwing + 2014-04-23 22:36:02 -0700 + Commit: 9716a72, github.com/apache/spark/pull/508 + + Mark all fields of EdgePartition, Graph, and GraphOps transient + Ankur Dave + 2014-04-23 22:01:13 -0700 + Commit: bdd2691, github.com/apache/spark/pull/520 + + Update Java api for setJobGroup with interruptOnCancel + Aaron Davidson + 2014-04-23 22:00:22 -0700 + Commit: 36511ea, github.com/apache/spark/pull/522 + + [Hot Fix #469] Fix flaky test in SparkListenerSuite + Andrew Or + 2014-04-23 21:59:33 -0700 + Commit: 99c0c33, github.com/apache/spark/pull/516 + + [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions. + Matei Zaharia + 2014-04-23 17:03:54 -0700 + Commit: 31c7d83, github.com/apache/spark/pull/487 + + SPARK-1582 Invoke Thread.interrupt() when cancelling jobs + Aaron Davidson + 2014-04-23 16:52:49 -0700 + Commit: 55e6bea, github.com/apache/spark/pull/498 + + Honor default fs name when initializing event logger. + Marcelo Vanzin + 2014-04-23 14:47:38 -0700 + Commit: 46b30f9, github.com/apache/spark/pull/450 + + SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent + Aaron Davidson + 2014-04-23 14:46:30 -0700 + Commit: be8f26f, github.com/apache/spark/pull/486 + + SPARK-1583: Fix a bug that using java.util.HashMap by mistake + zsxwing + 2014-04-23 14:12:20 -0700 + Commit: 19ef78f, github.com/apache/spark/pull/500 + + SPARK-1119 and other build improvements + Patrick Wendell + 2014-04-23 10:19:32 -0700 + Commit: d36d75c, github.com/apache/spark/pull/502 + + [SQL] SPARK-1571 Mistake in java example code + Michael Armbrust + 2014-04-22 22:19:32 -0700 + Commit: b0d8793, github.com/apache/spark/pull/496 + + SPARK-1494 Don't initialize classes loaded by MIMA excludes. + Michael Armbrust + 2014-04-22 21:56:15 -0700 + Commit: 18b1867, github.com/apache/spark/pull/494 + + SPARK-1562 Fix visibility / annotation of Spark SQL APIs + Michael Armbrust + 2014-04-22 20:02:33 -0700 + Commit: 0e03e6a, github.com/apache/spark/pull/489 + + [FIX: SPARK-1376] use --arg instead of --args in SparkSubmit to avoid warning messages + Xiangrui Meng + 2014-04-22 19:38:27 -0700 + Commit: 61930bd, github.com/apache/spark/pull/485 + + [streaming][SPARK-1578] Removed requirement for TTL in StreamingContext. + Tathagata Das + 2014-04-22 19:35:13 -0700 + Commit: bf47559, github.com/apache/spark/pull/491 + + [Spark-1538] Fix SparkUI incorrectly hiding persisted RDDs + Andrew Or + 2014-04-22 19:24:03 -0700 + Commit: 104590c, github.com/apache/spark/pull/469 + + Assorted clean-up for Spark-on-YARN. + Patrick Wendell + 2014-04-22 19:22:06 -0700 + Commit: f764f47, github.com/apache/spark/pull/488 + + [SPARK-1570] Fix classloading in JavaSQLContext.applySchema + Kan Zhang + 2014-04-22 15:05:12 -0700 + Commit: f9734e2, github.com/apache/spark/pull/484 + + Fix compilation on Hadoop 2.4.x. + Marcelo Vanzin + 2014-04-22 14:28:41 -0700 + Commit: b6ba546, github.com/apache/spark/pull/483 + + [Fix #204] Eliminate delay between binding and log checking + Andrew Or + 2014-04-22 14:27:49 -0700 + Commit: 54c96c2, github.com/apache/spark/pull/441 + + [Fix #274] Document + fix annotation usages + Andrew Or + 2014-04-21 22:24:44 -0700 + Commit: 898fc34, github.com/apache/spark/pull/470 + + [HOTFIX] SPARK-1399: remove outdated comments + CodingCat + 2014-04-22 09:43:13 -0700 + Commit: 61d7401, github.com/apache/spark/pull/474 + + [SPARK-1281] Improve partitioning in ALS + Tor Myklebust + 2014-04-22 11:07:30 -0700 + Commit: 4834adf, github.com/apache/spark/pull/407 + + fix bugs of dot in python + Xusen Yin + 2014-04-22 11:06:18 -0700 + Commit: 4f2f093, github.com/apache/spark/pull/463 + + [SPARK-1506][MLLIB] Documentation improvements for MLlib 1.0 + Xiangrui Meng + 2014-04-22 11:20:47 -0700 + Commit: 3f708f5, github.com/apache/spark/pull/422 + + [SPARK-1560]: Updated Pyrolite Dependency to be Java 6 compatible + Ahir Reddy + 2014-04-22 09:44:41 -0700 + Commit: 798d93f, github.com/apache/spark/pull/479 + + SPARK-1496: Have jarOfClass return Option[String] + Patrick Wendell + 2014-04-22 00:42:16 -0700 + Commit: 72aa131, github.com/apache/spark/pull/438 + + [SPARK-1459] Use local path (and not complete URL) when opening local lo... + Marcelo Vanzin + 2014-04-21 23:10:53 -0700 + Commit: 0a73103, github.com/apache/spark/pull/375 + + [SPARK-1439, SPARK-1440] Generate unified Scaladoc across projects and Javadocs + Matei Zaharia + 2014-04-21 21:57:40 -0700 + Commit: b0d70e4, github.com/apache/spark/pull/457 + + [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API [WIP] + Tathagata Das + 2014-04-21 19:04:49 -0700 + Commit: 94cbe23, github.com/apache/spark/pull/300 + + Updating versions for Spark 1.0 + Patrick Wendell + 2014-04-21 16:59:38 -0700 + Commit: a34e6fd + + Dev script: include RC name in git tag + Patrick Wendell + 2014-04-21 14:21:17 -0700 + Commit: 1532af9 + + SPARK-1399: show stage failure reason in UI + CodingCat , Nan Zhu + 2014-04-21 14:10:23 -0700 + Commit: 4b9220d, github.com/apache/spark/pull/421 + + SPARK-1539: RDDPage.scala contains RddPage class + Xiangrui Meng + 2014-04-21 12:48:02 -0700 + Commit: 8aa3860, github.com/apache/spark/pull/454 + + [Hot Fix] Ignore org.apache.spark.ui.UISuite tests + Andrew Or + 2014-04-21 12:37:43 -0700 + Commit: d62ce6d, github.com/apache/spark/pull/466 + + REPL cleanup. + Michael Armbrust + 2014-04-19 17:32:24 -0700 + Commit: 8e1e7ec, github.com/apache/spark/pull/451 + + Clean up and simplify Spark configuration + Patrick Wendell + 2014-04-21 10:26:33 -0700 + Commit: 29ee101, github.com/apache/spark/pull/299 + + [SPARK-1535] ALS: Avoid the garbage-creating ctor of DoubleMatrix + Tor Myklebust + 2014-04-19 15:10:18 -0700 + Commit: 9ce6ed4, github.com/apache/spark/pull/442 + + Add insertInto and saveAsTable to Python API. + Michael Armbrust + 2014-04-19 15:08:54 -0700 + Commit: 6ab0719, github.com/apache/spark/pull/447 + + Use scala deprecation instead of java. + Michael Armbrust + 2014-04-19 15:06:04 -0700 + Commit: bfb09c6, github.com/apache/spark/pull/452 + + README update + Reynold Xin + 2014-04-18 22:34:39 -0700 + Commit: 2fe6b18, github.com/apache/spark/pull/443 + + SPARK-1482: Fix potential resource leaks in saveAsHadoopDataset and save... + zsxwing + 2014-04-18 17:49:22 -0700 + Commit: ea17460, github.com/apache/spark/pull/400 + + SPARK-1456 Remove view bounds on Ordered in favor of a context bound on Ordering. + Michael Armbrust + 2014-04-18 12:04:13 -0700 + Commit: 9e21b97, github.com/apache/spark/pull/410 + + Fixed broken pyspark shell. + Reynold Xin + 2014-04-18 10:10:13 -0700 + Commit: d8767c4, github.com/apache/spark/pull/444 + + SPARK-1523: improve the readability of code in AkkaUtil + CodingCat + 2014-04-18 10:05:00 -0700 + Commit: 171cea8, github.com/apache/spark/pull/434 + + SPARK-1357 (addendum). More Experimental items in MLlib + Sean Owen + 2014-04-18 10:04:02 -0700 + Commit: 1a30429, github.com/apache/spark/pull/372 + + [SPARK-1520] remove fastutil from dependencies + Xiangrui Meng + 2014-04-18 10:03:15 -0700 + Commit: c40eec8, github.com/apache/spark/pull/437 + + Reuses Row object in ExistingRdd.productToRowRdd() + Cheng Lian + 2014-04-18 10:02:27 -0700 + Commit: 977467e, github.com/apache/spark/pull/432 + + SPARK-1483: Rename minSplits to minPartitions in public APIs + CodingCat + 2014-04-18 10:01:16 -0700 + Commit: 969a075, github.com/apache/spark/pull/430 + + HOTFIX: Ignore streaming UI test + Patrick Wendell + 2014-04-17 17:33:24 -0700 + Commit: 1c0dc37, github.com/apache/spark/pull/440 + + FIX: Don't build Hive in assembly unless running Hive tests. + Patrick Wendell + 2014-04-17 17:24:00 -0700 + Commit: 3225272, github.com/apache/spark/pull/439 + + Add clean to build + Patrick Wendell + 2014-04-16 16:32:34 -0700 + Commit: 67d01d8 + + HOTFIX: Use file name and not paths for excludes + Patrick Wendell + 2014-04-14 15:51:54 -0700 + Commit: 5349fab + + SPARK-1408 Modify Spark on Yarn to point to the history server when app ... + Thomas Graves + 2014-04-17 16:36:37 -0500 + Commit: 6195fb8, github.com/apache/spark/pull/362 + + [SPARK-1395] Allow "local:" URIs to work on Yarn. + Marcelo Vanzin + 2014-04-17 10:29:38 -0500 + Commit: a83a794, github.com/apache/spark/pull/303 + + [python alternative] pyspark require Python2, failing if system default is Py3 from shell.py + AbhishekKr + 2014-04-16 19:05:40 -0700 + Commit: b3ad707, github.com/apache/spark/pull/399 + + SPARK-1462: Examples of ML algorithms are using deprecated APIs + Sandeep + 2014-04-16 18:23:07 -0700 + Commit: 13fb4c7, github.com/apache/spark/pull/416 + + Include stack trace for exceptions thrown by user code. + Michael Armbrust + 2014-04-16 18:12:56 -0700 + Commit: aef8a4a, github.com/apache/spark/pull/409 + + Update ReducedWindowedDStream.scala + baishuo(白硕) + 2014-04-16 18:08:11 -0700 + Commit: 822353d, github.com/apache/spark/pull/425 + + misleading task number of groupByKey + Chen Chao + 2014-04-16 17:58:42 -0700 + Commit: 51c41da, github.com/apache/spark/pull/403 + + Fixing a race condition in event listener unit test + Kan Zhang + 2014-04-16 17:39:11 -0700 + Commit: f0abf5f, github.com/apache/spark/pull/401 + + remove unnecessary brace and semicolon in 'putBlockInfo.synchronize' block + Chen Chao + 2014-04-16 17:30:01 -0700 + Commit: e43e31d, github.com/apache/spark/pull/411 + + SPARK-1329: Create pid2vid with correct number of partitions + Ankur Dave + 2014-04-16 17:16:55 -0700 + Commit: b4ea3d9, github.com/apache/spark/pull/368 + + Rebuild routing table after Graph.reverse + Ankur Dave + 2014-04-16 17:15:50 -0700 + Commit: 602b9ea, github.com/apache/spark/pull/431 + + [SPARK-1511] use Files.move instead of renameTo in TestUtils.scala + Ye Xianjin + 2014-04-16 14:56:22 -0700 + Commit: 87a7c4f, github.com/apache/spark/pull/427 + + SPARK-1465: Spark compilation is broken with the latest hadoop-2.4.0 release + xuan , xuan + 2014-04-16 14:41:22 -0500 + Commit: d8fc4a4, github.com/apache/spark/pull/396 + + SPARK-1469: Scheduler mode should accept lower-case definitions and have... + Sandeep + 2014-04-16 09:58:57 -0700 + Commit: b75301f, github.com/apache/spark/pull/388 + + Minor addition to SPARK-1497 + Patrick Wendell + 2014-04-16 09:43:17 -0700 + Commit: 4479ecd + + SPARK-1497. Fix scalastyle warnings in YARN, Hive code + Sean Owen + 2014-04-16 09:34:59 -0700 + Commit: c744d66, github.com/apache/spark/pull/413 + + SPARK-1310: Start adding k-fold cross validation to MLLib [adds kFold to MLUtils & fixes bug in BernoulliSampler] + Holden Karau + 2014-04-16 09:33:27 -0700 + Commit: 8efec04, github.com/apache/spark/pull/18 + + update spark.default.parallelism + Chen Chao + 2014-04-16 09:14:18 -0700 + Commit: e4f5577, github.com/apache/spark/pull/389 + + Loads test tables when running "sbt hive/console" without HIVE_DEV_HOME + Cheng Lian + 2014-04-16 08:54:34 -0700 + Commit: 9e908ab, github.com/apache/spark/pull/417 + + Make "spark logo" link refer to "/". + Marcelo Vanzin + 2014-04-16 08:53:01 -0700 + Commit: 5fe18a7, github.com/apache/spark/pull/408 + + [SPARK-959] Updated SBT from 0.13.1 to 0.13.2 + Cheng Lian + 2014-04-16 08:52:14 -0700 + Commit: 1ea9a21, github.com/apache/spark/pull/426 + + [SQL] SPARK-1424 Generalize insertIntoTable functions on SchemaRDDs + Michael Armbrust + 2014-04-15 20:40:40 -0700 + Commit: e5130d9, github.com/apache/spark/pull/354 + + [WIP] SPARK-1430: Support sparse data in Python MLlib + Matei Zaharia + 2014-04-15 20:33:24 -0700 + Commit: 95647fa, github.com/apache/spark/pull/341 + + [FIX] update sbt-idea to version 1.6.0 + Xiangrui Meng + 2014-04-15 19:37:32 -0700 + Commit: 33d6e37, github.com/apache/spark/pull/419 + + SPARK-1455: Better isolation for unit tests. + Patrick Wendell + 2014-04-15 19:34:39 -0700 + Commit: 110e825, github.com/apache/spark/pull/420 + + Decision Tree documentation for MLlib programming guide + Manish Amde + 2014-04-15 11:14:28 -0700 + Commit: 194ed06, github.com/apache/spark/pull/402 + + [SPARK-1157][MLlib] L-BFGS Optimizer based on Breeze's implementation. + DB Tsai + 2014-04-15 11:12:47 -0700 + Commit: 5812472, github.com/apache/spark/pull/353 + + SPARK-1501: Ensure assertions in Graph.apply are asserted. + William Benton + 2014-04-15 10:38:42 -0700 + Commit: 692dd69, github.com/apache/spark/pull/415 + + SPARK-1426: Make MLlib work with NumPy versions older than 1.7 + Sandeep + 2014-04-15 00:19:43 -0700 + Commit: 1491b2a, github.com/apache/spark/pull/391 + + SPARK-1374: PySpark API for SparkSQL + Ahir Reddy , Michael Armbrust + 2014-04-15 00:07:55 -0700 + Commit: 7433f64, github.com/apache/spark/pull/363 + + SPARK-1488. Resolve scalac feature warnings during build + Sean Owen + 2014-04-14 19:50:00 -0700 + Commit: 7471828, github.com/apache/spark/pull/404 + + [BUGFIX] In-memory columnar storage bug fixes + Cheng Lian , Michael Armbrust + 2014-04-14 15:22:43 -0700 + Commit: fdebb69, github.com/apache/spark/pull/374 + + [SPARK-1415] Hadoop min split for wholeTextFiles() + Xusen Yin + 2014-04-13 13:18:52 -0700 + Commit: 1cf565f, github.com/apache/spark/pull/376 + + SPARK-1480: Clean up use of classloaders + Patrick Wendell + 2014-04-13 08:58:37 -0700 + Commit: 3537e25, github.com/apache/spark/pull/398 + + [SPARK-1403] Move the class loader creation back to where it was in 0.9.0 + Bharath Bhushan + 2014-04-12 20:52:29 -0700 + Commit: c970d86, github.com/apache/spark/pull/322 + + [Fix #204] Update out-dated comments + Andrew Or + 2014-04-12 16:33:38 -0700 + Commit: 52d401b, github.com/apache/spark/pull/381 + + [SPARK-1386] Web UI for Spark Streaming + Tathagata Das , Andrew Or + 2014-04-11 23:33:49 -0700 + Commit: f36dc3f, github.com/apache/spark/pull/290 + + SPARK-1057 (alternative) Remove fastutil + Sean Owen + 2014-04-11 22:46:47 -0700 + Commit: 4dfcb38, github.com/apache/spark/pull/266 + + Update WindowedDStream.scala + baishuo(白硕) + 2014-04-11 20:33:42 -0700 + Commit: dac6240, github.com/apache/spark/pull/390 + + [WIP] [SPARK-1328] Add vector statistics + Xusen Yin , Xiangrui Meng + 2014-04-11 19:43:22 -0700 + Commit: ce0ce3d, github.com/apache/spark/pull/268 + + [FIX] make coalesce test deterministic in RDDSuite + Xiangrui Meng + 2014-04-11 19:41:40 -0700 + Commit: 9afaeed, github.com/apache/spark/pull/387 + + HOTFIX: Ignore python metastore files in RAT checks. + Patrick Wendell + 2014-04-11 13:23:21 -0700 + Commit: 79eb276, github.com/apache/spark/pull/393 + + [SPARK-1225, 1241] [MLLIB] Add AreaUnderCurve and BinaryClassificationMetrics + Xiangrui Meng + 2014-04-11 12:06:13 -0700 + Commit: e6128b5, github.com/apache/spark/pull/364 + + Some clean up in build/docs + Patrick Wendell + 2014-04-11 10:45:27 -0700 + Commit: 170b09d, github.com/apache/spark/pull/382 + + SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken + Thomas Graves + 2014-04-11 13:17:48 +0530 + Commit: 9e90c46, github.com/apache/spark/pull/344 + + SPARK-1202: Improvements to task killing in the UI. + Patrick Wendell + 2014-04-10 20:43:56 -0700 + Commit: a1bb4c6, github.com/apache/spark/pull/386 + + Add Spark v0.9.1 to ec2 launch script and use it as the default + Harvey Feng + 2014-04-10 18:25:54 -0700 + Commit: 59de39b, github.com/apache/spark/pull/385 + + Set spark.executor.uri from environment variable (needed by Mesos) + Ivan Wick + 2014-04-10 17:49:30 -0700 + Commit: 41df293, github.com/apache/spark/pull/311 + + SPARK-1202 - Add a "cancel" button in the UI for stages + Sundeep Narravula , Sundeep Narravula + 2014-04-10 17:10:11 -0700 + Commit: 211f974, github.com/apache/spark/pull/246 + + [SQL] Improve column pruning in the optimizer. + Michael Armbrust + 2014-04-10 16:20:33 -0700 + Commit: 4843ef0, github.com/apache/spark/pull/378 + + Remove Unnecessary Whitespace's + Sandeep + 2014-04-10 15:04:13 -0700 + Commit: 09bf14b, github.com/apache/spark/pull/380 + + Update tuning.md + Andrew Ash + 2014-04-10 14:59:58 -0700 + Commit: 4c9906d, github.com/apache/spark/pull/384 + + Revert "SPARK-1433: Upgrade Mesos dependency to 0.17.0" + Patrick Wendell + 2014-04-10 14:43:29 -0700 + Commit: 1e2cdbc + + SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining + Sandeep + 2014-04-10 11:17:41 -0700 + Commit: 2ac43ad, github.com/apache/spark/pull/356 + + [SPARK-1276] Add a HistoryServer to render persisted UI + Andrew Or + 2014-04-10 10:39:34 -0700 + Commit: 9ae80bf, github.com/apache/spark/pull/204 + + Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL + witgo + 2014-04-10 10:35:24 -0700 + Commit: a74fbbb, github.com/apache/spark/pull/325 + + Revert "SPARK-729: Closures not always serialized at capture time" + Patrick Wendell + 2014-04-10 02:10:40 -0700 + Commit: e6d4a74 + + SPARK-1446: Spark examples should not do a System.exit + Sandeep + 2014-04-10 00:37:21 -0700 + Commit: e55cc4b, github.com/apache/spark/pull/370 + + SPARK-729: Closures not always serialized at capture time + William Benton + 2014-04-09 18:56:27 -0700 + Commit: 8ca3b2b, github.com/apache/spark/pull/189 + + [SPARK-1357 (fix)] remove empty line after :: DeveloperApi/Experimental :: + Xiangrui Meng + 2014-04-09 17:08:17 -0700 + Commit: 0adc932, github.com/apache/spark/pull/373 + + SPARK-1407 drain event queue before stopping event logger + Kan Zhang + 2014-04-09 15:24:33 -0700 + Commit: eb5f2b6, github.com/apache/spark/pull/366 + + [SPARK-1357] [MLLIB] Annotate developer and experimental APIs + Xiangrui Meng + 2014-04-09 02:21:15 -0700 + Commit: bde9cc1, github.com/apache/spark/pull/298 + + SPARK-1093: Annotate developer and experimental API's + Patrick Wendell , Andrew Or + 2014-04-09 01:14:46 -0700 + Commit: 87bd1f9, github.com/apache/spark/pull/274 + + [SPARK-1390] Refactoring of matrices backed by RDDs + Xiangrui Meng + 2014-04-08 23:01:15 -0700 + Commit: 9689b66, github.com/apache/spark/pull/296 + + Spark-939: allow user jars to take precedence over spark jars + Holden Karau + 2014-04-08 22:29:21 -0700 + Commit: fa0524f, github.com/apache/spark/pull/217 + + [SPARK-1434] [MLLIB] change labelParser from anonymous function to trait + Xiangrui Meng + 2014-04-08 20:37:01 -0700 + Commit: b9e0c93, github.com/apache/spark/pull/345 + + Spark 1271: Co-Group and Group-By should pass Iterable[X] + Holden Karau + 2014-04-08 18:15:52 -0700 + Commit: ce8ec54, github.com/apache/spark/pull/242 + + SPARK-1433: Upgrade Mesos dependency to 0.17.0 + Sandeep + 2014-04-08 16:19:22 -0700 + Commit: 12c077d, github.com/apache/spark/pull/355 + + [SPARK-1397] Notify SparkListeners when stages fail or are cancelled. + Kay Ousterhout + 2014-04-08 14:42:02 -0700 + Commit: fac6085, github.com/apache/spark/pull/309 + + SPARK-1445: compute-classpath should not print error if lib_managed not found + Aaron Davidson + 2014-04-08 14:40:20 -0700 + Commit: e25b593, github.com/apache/spark/pull/361 + + SPARK-1348 binding Master, Worker, and App Web UI to all interfaces + Kan Zhang + 2014-04-08 14:30:24 -0700 + Commit: a8d86b0, github.com/apache/spark/pull/318 + + Remove extra semicolon in import statement and unused import in ApplicationMaster + Henry Saputra + 2014-04-08 14:23:16 -0700 + Commit: 3bc0548, github.com/apache/spark/pull/358 + + [SPARK-1396] Properly cleanup DAGScheduler on job cancellation. + Kay Ousterhout + 2014-04-08 01:03:33 -0700 + Commit: 6dc5f58, github.com/apache/spark/pull/305 + + [SPARK-1331] Added graceful shutdown to Spark Streaming + Tathagata Das + 2014-04-08 00:00:17 -0700 + Commit: 83ac9a4, github.com/apache/spark/pull/247 + + [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast data + Tathagata Das , Andrew Or , Roman Pastukhov + 2014-04-07 23:40:21 -0700 + Commit: 11eabbe, github.com/apache/spark/pull/126 + + [SPARK-1402] Added 3 more compression schemes + Cheng Lian + 2014-04-07 22:24:12 -0700 + Commit: 0d0493f, github.com/apache/spark/pull/330 + + Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis). + Reynold Xin + 2014-04-07 19:28:24 -0700 + Commit: f27e56a, github.com/apache/spark/pull/352 + + Added eval for Rand (without any support for user-defined seed). + Reynold Xin + 2014-04-07 18:40:08 -0700 + Commit: 31e6fff, github.com/apache/spark/pull/349 + + Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched. + Reynold Xin + 2014-04-07 18:39:18 -0700 + Commit: 55dfd5d, github.com/apache/spark/pull/350 + + [sql] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables. + Reynold Xin + 2014-04-07 18:38:44 -0700 + Commit: 14c9238, github.com/apache/spark/pull/348 + + SPARK-1099: Introduce local[*] mode to infer number of cores + Aaron Davidson + 2014-04-07 13:06:30 -0700 + Commit: 0307db0, github.com/apache/spark/pull/182 + + HOTFIX: Disable actor input stream test. + Patrick Wendell + 2014-04-07 12:47:27 -0700 + Commit: 2a2ca48, github.com/apache/spark/pull/347 + + SPARK-1252. On YARN, use container-log4j.properties for executors + Sandy Ryza + 2014-04-07 13:28:14 -0500 + Commit: 9dd8b91, github.com/apache/spark/pull/148 + + [sql] Rename Expression.apply to eval for better readability. + Reynold Xin + 2014-04-07 10:45:31 -0700 + Commit: 83f2a2f, github.com/apache/spark/pull/340 + + SPARK-1432: Make sure that all metadata fields are properly cleaned + Davis Shepherd + 2014-04-07 10:02:00 -0700 + Commit: a3c51c6, github.com/apache/spark/pull/338 + + [SQL] SPARK-1427 Fix toString for SchemaRDD NativeCommands. + Michael Armbrust + 2014-04-07 01:46:50 -0700 + Commit: b5bae84, github.com/apache/spark/pull/343 + + [SQL] SPARK-1371 Hash Aggregation Improvements + Michael Armbrust + 2014-04-07 00:14:00 -0700 + Commit: accd099, github.com/apache/spark/pull/295 + + SPARK-1431: Allow merging conflicting pull requests + Patrick Wendell + 2014-04-06 21:04:45 -0700 + Commit: 87d0928, github.com/apache/spark/pull/342 + + SPARK-1154: Clean up app folders in worker nodes + Evan Chan , Kelvin Chu + 2014-04-06 19:17:33 -0700 + Commit: 1440154, github.com/apache/spark/pull/288 + + SPARK-1314: Use SPARK_HIVE to determine if we include Hive in packaging + Aaron Davidson + 2014-04-06 17:48:41 -0700 + Commit: 4106558, github.com/apache/spark/pull/237 + + SPARK-1349: spark-shell gets its own command history + Aaron Davidson + 2014-04-06 17:43:44 -0700 + Commit: 7ce52c4, github.com/apache/spark/pull/267 + + SPARK-1387. Update build plugins, avoid plugin version warning, centralize versions + Sean Owen + 2014-04-06 17:40:37 -0700 + Commit: 856c50f, github.com/apache/spark/pull/291 + + [SPARK-1259] Make RDD locally iterable + Egor Pakhomov + 2014-04-06 16:41:23 -0700 + Commit: e258e50, github.com/apache/spark/pull/156 + + Fix SPARK-1420 The maven build error for Spark Catalyst + witgo + 2014-04-06 16:03:06 -0700 + Commit: 7012ffa, github.com/apache/spark/pull/333 + + SPARK-1421. Make MLlib work on Python 2.6 + Matei Zaharia + 2014-04-05 20:52:05 -0700 + Commit: 0b85516, github.com/apache/spark/pull/335 + + Fix for PR #195 for Java 6 + Sean Owen + 2014-04-05 19:08:24 -0700 + Commit: 890d63b, github.com/apache/spark/pull/334 + + [SPARK-1371] fix computePreferredLocations signature to not depend on underlying implementation + Mridul Muralidharan + 2014-04-05 15:23:37 -0700 + Commit: 6e88583, github.com/apache/spark/pull/302 + + Remove the getStageInfo() method from SparkContext. + Kay Ousterhout + 2014-04-05 15:17:50 -0700 + Commit: 2d0150c, github.com/apache/spark/pull/308 + + HOTFIX for broken CI, by SPARK-1336 + Prashant Sharma , Prashant Sharma + 2014-04-04 22:49:19 -0700 + Commit: 7c18428, github.com/apache/spark/pull/321 + + small fix ( proogram -> program ) + Prabeesh K + 2014-04-04 21:32:00 -0700 + Commit: 0acc7a0, github.com/apache/spark/pull/331 + + [SQL] SPARK-1366 Consistent sql function across different types of SQLContexts + Michael Armbrust + 2014-04-04 21:15:33 -0700 + Commit: 8de038e, github.com/apache/spark/pull/319 + + SPARK-1305: Support persisting RDD's directly to Tachyon + Haoyuan Li , RongGu + 2014-04-04 20:36:24 -0700 + Commit: b50ddfd, github.com/apache/spark/pull/158 + + [SPARK-1419] Bumped parent POM to apache 14 + Mark Hamstra + 2014-04-04 19:19:48 -0700 + Commit: 1347ebd, github.com/apache/spark/pull/328 + + Add test utility for generating Jar files with compiled classes. + Patrick Wendell + 2014-04-04 19:15:15 -0700 + Commit: 5f3c1bb, github.com/apache/spark/pull/326 + + SPARK-1414. Python API for SparkContext.wholeTextFiles + Matei Zaharia + 2014-04-04 17:29:29 -0700 + Commit: 60e18ce, github.com/apache/spark/pull/327 + + [SQL] Minor fixes. + Michael Armbrust + 2014-04-04 17:23:17 -0700 + Commit: d956cc2, github.com/apache/spark/pull/315 + + [SPARK-1198] Allow pipes tasks to run in different sub-directories + Thomas Graves + 2014-04-04 17:16:31 -0700 + Commit: 198892f, github.com/apache/spark/pull/128 + + Don't create SparkContext in JobProgressListenerSuite. + Patrick Wendell + 2014-04-04 14:46:32 -0700 + Commit: a02b535, github.com/apache/spark/pull/324 + + SPARK-1375. Additional spark-submit cleanup + Sandy Ryza + 2014-04-04 13:28:42 -0700 + Commit: 16b8308, github.com/apache/spark/pull/278 + + [SPARK-1133] Add whole text files reader in MLlib + Xusen Yin + 2014-04-04 11:12:47 -0700 + Commit: f1fa617, github.com/apache/spark/pull/252 + + SPARK-1404: Always upgrade spark-env.sh vars to environment vars + Aaron Davidson + 2014-04-04 09:50:24 -0700 + Commit: 01cf4c4, github.com/apache/spark/pull/310 + + SPARK-1350. Always use JAVA_HOME to run executor container JVMs. + Sandy Ryza + 2014-04-04 08:54:04 -0500 + Commit: 7f32fd4, github.com/apache/spark/pull/313 + + SPARK-1337: Application web UI garbage collects newest stages + Patrick Wendell + 2014-04-03 22:13:56 -0700 + Commit: ee6e9e7, github.com/apache/spark/pull/320 + + Revert "[SPARK-1398] Removed findbugs jsr305 dependency" + Patrick Wendell + 2014-04-03 17:00:06 -0700 + Commit: 33e6361 + + Fix jenkins from giving the green light to builds that don't compile. + Michael Armbrust + 2014-04-03 16:53:35 -0700 + Commit: 9231b01, github.com/apache/spark/pull/317 + + [BUILD FIX] Fix compilation of Spark SQL Java API. + Michael Armbrust + 2014-04-03 16:12:08 -0700 + Commit: d94826b, github.com/apache/spark/pull/316 + + [SPARK-1134] Fix and document passing of arguments to IPython + Diana Carroll + 2014-04-03 15:48:42 -0700 + Commit: a599e43, github.com/apache/spark/pull/294 + + [SQL] SPARK-1333 First draft of java API + Michael Armbrust + 2014-04-03 15:45:34 -0700 + Commit: b8f5341, github.com/apache/spark/pull/248 + + Spark 1162 Implemented takeOrdered in pyspark. + Prashant Sharma + 2014-04-03 15:42:17 -0700 + Commit: c1ea3af, github.com/apache/spark/pull/97 + + [SPARK-1360] Add Timestamp Support for SQL + Cheng Hao + 2014-04-03 15:33:17 -0700 + Commit: 5d1feda, github.com/apache/spark/pull/275 + + Spark parquet improvements + Andre Schumacher + 2014-04-03 15:31:47 -0700 + Commit: fbebaed, github.com/apache/spark/pull/195 + + [SPARK-1398] Removed findbugs jsr305 dependency + Mark Hamstra + 2014-04-03 14:08:47 -0700 + Commit: 92a86b2, github.com/apache/spark/pull/307 + + [SQL] SPARK-1364 Improve datatype and test coverage for ScalaReflection schema inference. + Michael Armbrust + 2014-04-02 18:14:31 -0700 + Commit: 47ebea5, github.com/apache/spark/pull/293 + + [SPARK-1212, Part II] Support sparse data in MLlib + Xiangrui Meng + 2014-04-02 14:01:12 -0700 + Commit: 9c65fa7, github.com/apache/spark/pull/245 + + StopAfter / TopK related changes + Reynold Xin , Michael Armbrust + 2014-04-02 12:48:04 -0700 + Commit: ed730c9, github.com/apache/spark/pull/233 + + [SPARK-1371][WIP] Compression support for Spark SQL in-memory columnar storage + Cheng Lian + 2014-04-02 12:47:22 -0700 + Commit: 1faa579, github.com/apache/spark/pull/285 + + Do not re-use objects in the EdgePartition/EdgeTriplet iterators. + Daniel Darabos + 2014-04-02 12:27:37 -0700 + Commit: 7823633, github.com/apache/spark/pull/276 + + [SPARK-1385] Use existing code for JSON de/serialization of BlockId + Andrew Or + 2014-04-02 10:43:09 -0700 + Commit: de8eefa, github.com/apache/spark/pull/289 + + Renamed stageIdToActiveJob to jobIdToActiveJob. + Kay Ousterhout + 2014-04-02 10:35:52 -0700 + Commit: 11973a7, github.com/apache/spark/pull/301 + + Remove * from test case golden filename. + Michael Armbrust + 2014-04-01 23:54:38 -0700 + Commit: ea9de65, github.com/apache/spark/pull/297 + + MLI-1 Decision Trees + Manish Amde , manishamde , Xiangrui Meng + 2014-04-01 21:40:49 -0700 + Commit: 8b3045c, github.com/apache/spark/pull/79 + + Revert "[Spark-1134] only call ipython if no arguments are given; remove IPYTHONOPTS from call" + Matei Zaharia + 2014-04-01 19:31:50 -0700 + Commit: 45df912 + + [Spark-1134] only call ipython if no arguments are given; remove IPYTHONOPTS from call + Diana Carroll + 2014-04-01 19:29:26 -0700 + Commit: afb5ea6, github.com/apache/spark/pull/227 + + [SPARK-1342] Scala 2.10.4 + Mark Hamstra + 2014-04-01 18:35:50 -0700 + Commit: 764353d, github.com/apache/spark/pull/259 + + [SQL] SPARK-1372 Support for caching and uncaching tables in a SQLContext. + Michael Armbrust + 2014-04-01 14:45:44 -0700 + Commit: f5c418d, github.com/apache/spark/pull/282 + + [Hot Fix #42] Persisted RDD disappears on storage page if re-used + Andrew Or + 2014-03-31 23:01:14 -0700 + Commit: ada310a, github.com/apache/spark/pull/281 + + [SPARK-1377] Upgrade Jetty to 8.1.14v20131031 + Andrew Or + 2014-03-31 21:42:36 -0700 + Commit: 94fe7fd, github.com/apache/spark/pull/280 + + SPARK-1376. In the yarn-cluster submitter, rename "args" option to "arg" + Sandy Ryza + 2014-04-01 08:26:31 +0530 + Commit: 564f1c1, github.com/apache/spark/pull/279 + + SPARK-1365 [HOTFIX] Fix RateLimitedOutputStream test + Patrick Wendell + 2014-03-31 16:25:43 -0700 + Commit: 33b3c2a, github.com/apache/spark/pull/277 + + [SQL] Rewrite join implementation to allow streaming of one relation. + Michael Armbrust + 2014-03-31 15:23:46 -0700 + Commit: 5731af5, github.com/apache/spark/pull/250 + + SPARK-1352: Improve robustness of spark-submit script + Patrick Wendell + 2014-03-31 12:07:14 -0700 + Commit: 841721e, github.com/apache/spark/pull/271 + + SPARK-1352 - Comment style single space before ending */ check. + Prashant Sharma + 2014-03-30 10:06:56 -0700 + Commit: d666053, github.com/apache/spark/pull/261 + + [SPARK-1354][SQL] Add tableName as a qualifier for SimpleCatelogy + jerryshao + 2014-03-30 10:03:58 -0700 + Commit: 95d7d2a, github.com/apache/spark/pull/272 + + SPARK-1336 Reducing the output of run-tests script. + Prashant Sharma , Prashant Sharma + 2014-03-29 23:03:03 -0700 + Commit: df1b9f7, github.com/apache/spark/pull/262 + + [SQL] SPARK-1354 Fix self-joins of parquet relations + Michael Armbrust + 2014-03-29 22:02:53 -0700 + Commit: 2861b07, github.com/apache/spark/pull/269 + + Don't swallow all kryo errors, only those that indicate we are out of data. + Michael Armbrust + 2014-03-29 22:01:29 -0700 + Commit: 92b8395, github.com/apache/spark/pull/142 + + [SPARK-1186] : Enrich the Spark Shell to support additional arguments. + Bernardo Gomez Palacio + 2014-03-29 19:49:22 -0700 + Commit: fda86d8, github.com/apache/spark/pull/116 + + Implement the RLike & Like in catalyst + Cheng Hao + 2014-03-29 15:12:43 -0700 + Commit: af3746c, github.com/apache/spark/pull/224 + + SPARK-1126. spark-app preliminary + Sandy Ryza + 2014-03-29 14:41:36 -0700 + Commit: 1617816, github.com/apache/spark/pull/86 + + SPARK-1345 adding missing dependency on avro for hadoop 0.23 to the new ... + Thomas Graves + 2014-03-28 23:09:29 -0700 + Commit: 3738f24, github.com/apache/spark/pull/263 + + fix path for jar, make sed actually work on OSX + Nick Lanham + 2014-03-28 13:33:35 -0700 + Commit: 75d46be, github.com/apache/spark/pull/264 + + SPARK-1096, a space after comment start style checker. + Prashant Sharma + 2014-03-28 00:21:49 -0700 + Commit: 60abc25, github.com/apache/spark/pull/124 + + Make sed do -i '' on OSX + Nick Lanham + 2014-03-27 22:45:00 -0700 + Commit: 632c322, github.com/apache/spark/pull/258 + + [SPARK-1210] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executo... + Takuya UESHIN + 2014-03-27 22:17:15 -0700 + Commit: 3d89043, github.com/apache/spark/pull/15 + + [SPARK-1268] Adding XOR and AND-NOT operations to spark.util.collection.BitSet + Petko Nikolov + 2014-03-27 15:49:07 -0700 + Commit: 6f986f0, github.com/apache/spark/pull/172 + + SPARK-1335. Also increase perm gen / code cache for scalatest when invoked via Maven build + Sean Owen + 2014-03-27 11:49:11 -0700 + Commit: 53953d0, github.com/apache/spark/pull/253 + + SPARK-1330 removed extra echo from comput_classpath.sh + Thomas Graves + 2014-03-27 11:54:43 -0500 + Commit: 426042a, github.com/apache/spark/pull/241 + + Cut down the granularity of travis tests. + Michael Armbrust + 2014-03-27 08:53:42 -0700 + Commit: 5b2d863, github.com/apache/spark/pull/255 + + [SPARK-1327] GLM needs to check addIntercept for intercept and weights + Xiangrui Meng + 2014-03-26 19:30:20 -0700 + Commit: d679843, github.com/apache/spark/pull/236 + + SPARK-1325. The maven build error for Spark Tools + Sean Owen , witgo + 2014-03-26 18:31:52 -0700 + Commit: 1fa48d9, github.com/apache/spark/pull/240 + + Spark 1095 : Adding explicit return types to all public methods + NirmalReddy , NirmalReddy + 2014-03-26 18:24:55 -0700 + Commit: 3e63d98, github.com/apache/spark/pull/168 + + SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS + Patrick Wendell + 2014-03-26 18:22:15 -0700 + Commit: be6d96c, github.com/apache/spark/pull/231 + + [SQL] Add a custom serializer for maps since they do not have a no-arg constructor. + Michael Armbrust + 2014-03-26 18:19:49 -0700 + Commit: e15e574, github.com/apache/spark/pull/243 + + [SQL] Un-ignore a test that is now passing. + Michael Armbrust + 2014-03-26 18:19:15 -0700 + Commit: 32cbdfd, github.com/apache/spark/pull/244 + + Unified package definition format in Spark SQL + Cheng Lian + 2014-03-26 15:36:18 -0700 + Commit: 345825d, github.com/apache/spark/pull/225 + + SPARK-1322, top in pyspark should sort result in descending order. + Prashant Sharma + 2014-03-26 09:16:37 -0700 + Commit: a0853a3, github.com/apache/spark/pull/235 + + SPARK-1321 Use Guava's top k implementation rather than our BoundedPriorityQueue based implementation + Reynold Xin + 2014-03-26 00:09:44 -0700 + Commit: b859853, github.com/apache/spark/pull/229 + + Initial experimentation with Travis CI configuration + Michael Armbrust + 2014-03-25 19:01:18 -0700 + Commit: 4f7d547, github.com/apache/spark/pull/230 + + Avoid Option while generating call site + witgo , Aaron Davidson + 2014-03-25 13:28:13 -0700 + Commit: 8237df8, github.com/apache/spark/pull/222 + + SPARK-1319: Fix scheduler to account for tasks using > 1 CPUs. + Shivaram Venkataraman + 2014-03-25 13:05:30 -0700 + Commit: f8111ea, github.com/apache/spark/pull/219 + + SPARK-1316. Remove use of Commons IO + Sean Owen + 2014-03-25 10:21:25 -0700 + Commit: 71d4ed2, github.com/apache/spark/pull/226 + + Add more hive compatability tests to whitelist + Michael Armbrust + 2014-03-25 09:57:26 -0700 + Commit: 134ace7, github.com/apache/spark/pull/220 + + SPARK-1286: Make usage of spark-env.sh idempotent + Aaron Davidson + 2014-03-24 22:24:21 -0700 + Commit: 007a733, github.com/apache/spark/pull/184 + + Unify the logic for column pruning, projection, and filtering of table scans. + Michael Armbrust + 2014-03-24 22:15:51 -0700 + Commit: b637f2d, github.com/apache/spark/pull/213 + + SPARK-1128: set hadoop task properties when constructing HadoopRDD + CodingCat , Nan Zhu + 2014-03-24 21:55:03 -0700 + Commit: 5140598, github.com/apache/spark/pull/101 + + SPARK-1094 Support MiMa for reporting binary compatibility accross versions. + Patrick Wendell , Prashant Sharma , Prashant Sharma + 2014-03-24 21:20:23 -0700 + Commit: dc126f2, github.com/apache/spark/pull/207 + + SPARK-1294 Fix resolution of uppercase field names using a HiveContext. + Michael Armbrust + 2014-03-24 19:24:22 -0700 + Commit: 8043b7b, github.com/apache/spark/pull/202 + + HOT FIX: Exclude test files from RAT + Patrick Wendell + 2014-03-24 13:38:07 -0700 + Commit: 56db8a2 + + SPARK-1144 Added license and RAT to check licenses. + Prashant Sharma + 2014-03-24 08:44:12 -0700 + Commit: 21109fb, github.com/apache/spark/pull/125 + + [SPARK-1212] Adding sparse data support and update KMeans + Xiangrui Meng + 2014-03-23 17:34:02 -0700 + Commit: 80c2968, github.com/apache/spark/pull/117 + + Fixed coding style issues in Spark SQL + Cheng Lian + 2014-03-23 15:21:40 -0700 + Commit: 8265dc7, github.com/apache/spark/pull/208 + + [SPARK-1292] In-memory columnar representation for Spark SQL + Cheng Lian , Cheng Lian + 2014-03-23 12:08:55 -0700 + Commit: 57a4379, github.com/apache/spark/pull/205 + + SPARK-1254. Supplemental fix for HTTPS on Maven Central + Sean Owen + 2014-03-23 10:57:01 -0700 + Commit: abf6714, github.com/apache/spark/pull/209 + + Fix to Stage UI to display numbers on progress bar + Emtiaz Ahmed + 2014-03-21 18:05:53 -0700 + Commit: 646e554, github.com/apache/spark/pull/201 + + Add asCode function for dumping raw tree representations. + Michael Armbrust + 2014-03-21 16:54:06 -0700 + Commit: d780983, github.com/apache/spark/pull/200 + + Make SQL keywords case-insensitive + Matei Zaharia + 2014-03-21 16:53:18 -0700 + Commit: dab5439, github.com/apache/spark/pull/193 + + SPARK-1279: Fix improper use of SimpleDateFormat + zsxwing + 2014-03-21 16:07:22 -0700 + Commit: 2c0aa22, github.com/apache/spark/pull/179 + + Add hive test files to repository. Remove download script. + Michael Armbrust + 2014-03-21 15:05:45 -0700 + Commit: 7e17fe6, github.com/apache/spark/pull/199 + + Fix maven jenkins: Add explicit init for required tables in SQLQuerySuite + Michael Armbrust + 2014-03-20 22:31:11 -0700 + Commit: e09139d, github.com/apache/spark/pull/191 + + SPARK-1251 Support for optimizing and executing structured queries + Michael Armbrust , Yin Huai , Reynold Xin , Lian, Cheng , Andre Schumacher , Yin Huai , Timothy Chen , Cheng Lian , Timothy Chen , Henry Cook , Mark Hamstra + 2014-03-20 18:03:20 -0700 + Commit: 9aadcff, github.com/apache/spark/pull/146 + + [Hot Fix #42] Do not stop SparkUI if bind() is not called + Andrew Or + 2014-03-20 14:13:16 -0700 + Commit: ca76423, github.com/apache/spark/pull/188 + + Principal Component Analysis + Reza Zadeh + 2014-03-20 10:39:20 -0700 + Commit: 66a03e5, github.com/apache/spark/pull/88 + + Revert "SPARK-1099:Spark's local mode should probably respect spark.cores.max by default" + Aaron Davidson + 2014-03-19 17:56:48 -0700 + Commit: ffe272d + + SPARK-1099:Spark's local mode should probably respect spark.cores.max by default + qqsun8819 + 2014-03-19 16:33:54 -0700 + Commit: 1678931, github.com/apache/spark/pull/110 + + Added doctest for map function in rdd.py + Jyotiska NK + 2014-03-19 14:04:45 -0700 + Commit: 67fa71c, github.com/apache/spark/pull/177 + + [SPARK-1132] Persisting Web UI through refactoring the SparkListener interface + Andrew Or , andrewor14 + 2014-03-19 13:17:01 -0700 + Commit: 79d07d6, github.com/apache/spark/pull/42 + + Bugfixes/improvements to scheduler + Mridul Muralidharan + 2014-03-19 12:46:55 -0700 + Commit: ab747d3, github.com/apache/spark/pull/159 + + SPARK-1203 fix saving to hdfs from yarn + Thomas Graves + 2014-03-19 08:09:20 -0500 + Commit: 6112270, github.com/apache/spark/pull/173 + + bugfix: Wrong "Duration" in "Active Stages" in stages page + shiyun.wxm + 2014-03-19 01:42:34 -0700 + Commit: d55ec86, github.com/apache/spark/pull/170 + + Bundle tachyon: SPARK-1269 + Nick Lanham + 2014-03-18 22:04:57 -0700 + Commit: a18ea00, github.com/apache/spark/pull/137 + + Fix SPARK-1256: Master web UI and Worker web UI returns a 404 error + witgo + 2014-03-18 21:57:47 -0700 + Commit: cc2655a, github.com/apache/spark/pull/150 + + [SPARK-1266] persist factors in implicit ALS + Xiangrui Meng + 2014-03-18 17:20:42 -0700 + Commit: f9d8a83, github.com/apache/spark/pull/165 + + [SPARK-1260]: faster construction of features with intercept + Xiangrui Meng + 2014-03-18 15:14:13 -0700 + Commit: e108b9a, github.com/apache/spark/pull/161 + + Update copyright year in NOTICE to 2014 + Matei Zaharia + 2014-03-18 14:34:31 -0700 + Commit: 79e547f, github.com/apache/spark/pull/174 + + SPARK-1102: Create a saveAsNewAPIHadoopDataset method + CodingCat + 2014-03-18 11:06:18 -0700 + Commit: 2fa26ec, github.com/apache/spark/pull/12 + + Revert "SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225." + Patrick Wendell + 2014-03-18 00:46:03 -0700 + Commit: e7423d4, github.com/apache/spark/pull/167 + + Spark 1246 add min max to stat counter + Dan McClary + 2014-03-18 00:45:47 -0700 + Commit: e3681f2, github.com/apache/spark/pull/144 + + [Spark-1261] add instructions for running python examples to doc overview page + Diana Carroll + 2014-03-17 17:35:51 -0700 + Commit: 087eedc, github.com/apache/spark/pull/162 + + SPARK-1244: Throw exception if map output status exceeds frame size + Patrick Wendell , Andrew Or + 2014-03-17 14:03:32 -0700 + Commit: 796977a, github.com/apache/spark/pull/152 + + SPARK-1240: handle the case of empty RDD when takeSample + CodingCat + 2014-03-16 22:14:59 -0700 + Commit: dc96546, github.com/apache/spark/pull/135 + + SPARK-1255: Allow user to pass Serializer object instead of class name for shuffle. + Reynold Xin + 2014-03-16 09:57:21 -0700 + Commit: f5486e9, github.com/apache/spark/pull/149 + + SPARK-1254. Consolidate, order, and harmonize repository declarations in Maven/SBT builds + Sean Owen + 2014-03-15 16:43:27 -0700 + Commit: 97e4459, github.com/apache/spark/pull/145 + + Fix serialization of MutablePair. Also provide an interface for easy updating. + Michael Armbrust + 2014-03-14 11:40:26 -0700 + Commit: e19044c, github.com/apache/spark/pull/141 + + [bugfix] wrong client arg, should use executor-cores + Tianshuo Deng + 2014-03-13 20:27:36 -0700 + Commit: 181b130, github.com/apache/spark/pull/138 + + SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225. + Reynold Xin + 2014-03-13 12:16:04 -0700 + Commit: ca4bf8c, github.com/apache/spark/pull/113 + + SPARK-1183. Don't use "worker" to mean executor + Sandy Ryza + 2014-03-13 12:11:33 -0700 + Commit: 6983732, github.com/apache/spark/pull/120 + + [SPARK-1237, 1238] Improve the computation of YtY for implicit ALS + Xiangrui Meng + 2014-03-13 00:43:19 -0700 + Commit: e4e8d8f, github.com/apache/spark/pull/131 + + SPARK-1019: pyspark RDD take() throws an NPE + Patrick Wendell + 2014-03-12 23:16:59 -0700 + Commit: 4ea23db, github.com/apache/spark/pull/112 + + hot fix for PR105 - change to Java annotation + CodingCat + 2014-03-12 19:49:18 -0700 + Commit: 6bd2eaa, github.com/apache/spark/pull/133 + + Fix example bug: compile error + jianghan + 2014-03-12 19:46:12 -0700 + Commit: 31a7040, github.com/apache/spark/pull/132 + + SPARK-1160: Deprecate toArray in RDD + CodingCat + 2014-03-12 17:43:12 -0700 + Commit: 9032f7c, github.com/apache/spark/pull/105 + + SPARK-1162 Added top in python. + Prashant Sharma + 2014-03-12 15:57:44 -0700 + Commit: b8afe30, github.com/apache/spark/pull/93 + + Fix #SPARK-1149 Bad partitioners can cause Spark to hang + liguoqiang + 2014-03-12 12:59:51 -0700 + Commit: 5d1ec64, github.com/apache/spark/pull/44 + + [SPARK-1233] Fix running hadoop 0.23 due to java.lang.NoSuchFieldException: DEFAULT_M... + Thomas Graves + 2014-03-12 11:25:41 -0700 + Commit: b5162f4, github.com/apache/spark/pull/129 + + [SPARK-1232] Fix the hadoop 0.23 yarn build + Thomas Graves + 2014-03-12 10:32:01 -0700 + Commit: c8c59b3, github.com/apache/spark/pull/127 + + Spark-1163, Added missing Python RDD functions + prabinb + 2014-03-11 23:57:05 -0700 + Commit: af7f2f1, github.com/apache/spark/pull/92 + + SPARK-1064 + Sandy Ryza + 2014-03-11 22:39:17 -0700 + Commit: 2409af9, github.com/apache/spark/pull/102 + + SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues... + Patrick Wendell + 2014-03-11 11:16:59 -0700 + Commit: 16788a6, github.com/apache/spark/pull/108 + + SPARK-1211. In ApplicationMaster, set spark.master system property to "y... + Sandy Ryza + 2014-03-10 17:42:33 -0700 + Commit: 2a2c964, github.com/apache/spark/pull/118 + + SPARK-1205: Clean up callSite/origin/generator. + Patrick Wendell + 2014-03-10 16:28:41 -0700 + Commit: 2a51617, github.com/apache/spark/pull/106 + + SPARK-1168, Added foldByKey to pyspark. + Prashant Sharma + 2014-03-10 13:37:11 -0700 + Commit: a59419c, github.com/apache/spark/pull/115 + + [SPARK-972] Added detailed callsite info for ValueError in context.py (resubmitted) + jyotiska + 2014-03-10 13:34:49 -0700 + Commit: f551898, github.com/apache/spark/pull/34 + + SPARK-977 Added Python RDD.zip function + Prabin Banka + 2014-03-10 13:27:00 -0700 + Commit: e1e09e0, github.com/apache/spark/pull/76 + + maintain arbitrary state data for each key + Chen Chao + 2014-03-09 22:42:12 -0700 + Commit: 5d98cfc, github.com/apache/spark/pull/114 + + SPARK-782 Clean up for ASM dependency. + Patrick Wendell + 2014-03-09 13:17:07 -0700 + Commit: b9be160, github.com/apache/spark/pull/100 + + Fix markup errors introduced in #33 (SPARK-1189) + Patrick Wendell + 2014-03-09 11:57:06 -0700 + Commit: faf4cad, github.com/apache/spark/pull/111 + + Add timeout for fetch file + Jiacheng Guo + 2014-03-09 11:37:44 -0700 + Commit: f6f9d02, github.com/apache/spark/pull/98 + + SPARK-929: Fully deprecate usage of SPARK_MEM + Aaron Davidson + 2014-03-09 11:08:39 -0700 + Commit: 52834d7, github.com/apache/spark/pull/99 + + SPARK-1190: Do not initialize log4j if slf4j log4j backend is not being used + Patrick Wendell + 2014-03-08 16:02:42 -0800 + Commit: e59a3b6, github.com/apache/spark/pull/107 + + Update junitxml plugin to the latest version to avoid recompilation in every SBT command. + Reynold Xin + 2014-03-08 12:40:26 -0800 + Commit: c2834ec, github.com/apache/spark/pull/104 + + [SPARK-1194] Fix the same-RDD rule for cache replacement + Cheng Lian + 2014-03-07 23:26:46 -0800 + Commit: 0b7b7fd, github.com/apache/spark/pull/96 + + Allow sbt to use more than 1G of heap. + Reynold Xin + 2014-03-07 23:23:59 -0800 + Commit: 8ad486a, github.com/apache/spark/pull/103 + + SPARK-1193. Fix indentation in pom.xmls + Sandy Ryza + 2014-03-07 23:10:35 -0800 + Commit: a99fb37, github.com/apache/spark/pull/91 + + Spark 1165 rdd.intersection in python and java + Prashant Sharma , Prashant Sharma + 2014-03-07 18:48:07 -0800 + Commit: 6e730ed, github.com/apache/spark/pull/80 + + SPARK-1195: set map_input_file environment variable in PipedRDD + Thomas Graves + 2014-03-07 10:36:55 -0800 + Commit: b7cd9e9, github.com/apache/spark/pull/94 + + SPARK-1136: Fix FaultToleranceTest for Docker 0.8.1 + Aaron Davidson + 2014-03-07 10:22:27 -0800 + Commit: dabeb6f, github.com/apache/spark/pull/5 + + Small clean-up to flatmap tests + Patrick Wendell + 2014-03-06 17:57:31 -0800 + Commit: 33baf14 + + Example for cassandra CQL read/write from spark + anitatailor + 2014-03-06 17:46:43 -0800 + Commit: 9ae919c, github.com/apache/spark/pull/87 + + SPARK-1197. Change yarn-standalone to yarn-cluster and fix up running on YARN docs + Sandy Ryza + 2014-03-06 17:12:58 -0800 + Commit: 328c73d, github.com/apache/spark/pull/95 + + SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets + Thomas Graves + 2014-03-06 18:27:50 -0600 + Commit: 7edbea4, github.com/apache/spark/pull/33 + + SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used + Kyle Ellrott + 2014-03-06 14:51:00 -0800 + Commit: 40566e1, github.com/apache/spark/pull/50 + + SPARK-1187, Added missing Python APIs + Prabin Banka + 2014-03-06 12:45:27 -0800 + Commit: 3d3acef, github.com/apache/spark/pull/75 + + SPARK-1156: allow user to login into a cluster without slaves + CodingCat + 2014-03-05 21:47:34 -0800 + Commit: 3eb009f, github.com/apache/spark/pull/58 + + SPARK-1184: Update the distribution tar.gz to include spark-assembly jar + Mark Grover + 2014-03-05 16:52:58 -0800 + Commit: cda381f, github.com/apache/spark/pull/78 + + Improve building with maven docs + liguoqiang + 2014-03-05 16:38:43 -0800 + Commit: 51ca7bd, github.com/apache/spark/pull/70 + + SPARK-1171: when executor is removed, we should minus totalCores instead of just freeCores on that executor + CodingCat , Nan Zhu + 2014-03-05 14:00:28 -0800 + Commit: a3da508, github.com/apache/spark/pull/63 + + SPARK-1109 wrong API docs for pyspark map function + Prashant Sharma + 2014-03-04 15:32:43 -0800 + Commit: 0283665, github.com/apache/spark/pull/73 + + SPARK-1178: missing document of spark.scheduler.revive.interval + CodingCat + 2014-03-04 10:28:17 -0800 + Commit: 1865dd6, github.com/apache/spark/pull/74 + + SPARK-1164 Deprecated reduceByKeyToDriver as it is an alias for reduceByKeyLocally + Prashant Sharma + 2014-03-04 10:27:02 -0800 + Commit: 2d8e0a0, github.com/apache/spark/pull/72 + + [java8API] SPARK-964 Investigate the potential for using JDK 8 lambda expressions for the Java/Scala APIs + Prashant Sharma , Patrick Wendell + 2014-03-03 22:31:30 -0800 + Commit: 181ec50, github.com/apache/spark/pull/17 + + Remove broken/unused Connection.getChunkFIFO method. + Kay Ousterhout + 2014-03-03 21:27:18 -0800 + Commit: b14ede7, github.com/apache/spark/pull/69 + + SPARK-1158: Fix flaky RateLimitedOutputStreamSuite. + Reynold Xin + 2014-03-03 21:24:19 -0800 + Commit: f5ae38a, github.com/apache/spark/pull/55 + + Added a unit test for PairRDDFunctions.lookup + Bryn Keller + 2014-03-03 16:38:57 -0800 + Commit: 923dba5, github.com/apache/spark/pull/36 + + Remove the remoteFetchTime metric. + Kay Ousterhout + 2014-03-03 16:12:00 -0800 + Commit: b55cade, github.com/apache/spark/pull/62 + + update proportion of memory + Chen Chao + 2014-03-03 14:41:25 -0800 + Commit: 9d225a9, github.com/apache/spark/pull/66 + + Removed accidentally checked in comment + Kay Ousterhout + 2014-03-03 14:39:49 -0800 + Commit: 369aad6, github.com/apache/spark/pull/61 + + SPARK-1173. (#2) Fix typo in Java streaming example. + Aaron Kimball + 2014-03-02 23:48:48 -0800 + Commit: f65c1f3, github.com/apache/spark/pull/65 + + SPARK-1173. Improve scala streaming docs. + Aaron Kimball + 2014-03-02 23:26:47 -0800 + Commit: 2b53447, github.com/apache/spark/pull/64 + + Add Jekyll tag to isolate "production-only" doc components. + Patrick Wendell + 2014-03-02 18:19:01 -0800 + Commit: 55a4f11, github.com/apache/spark/pull/56 + + SPARK-1121: Include avro for yarn-alpha builds + Patrick Wendell + 2014-03-02 15:18:19 -0800 + Commit: c3f5e07, github.com/apache/spark/pull/49 + + SPARK-1084.2 (resubmitted) + Sean Owen + 2014-03-02 14:27:53 -0800 + Commit: fd31adb, github.com/apache/spark/pull/32 + + Ignore RateLimitedOutputStreamSuite for now. + Reynold Xin + 2014-03-02 14:27:19 -0800 + Commit: 353ac6b, github.com/apache/spark/pull/54 + + SPARK-1137: Make ZK PersistenceEngine not crash for wrong serialVersionUID + Aaron Davidson + 2014-03-02 01:00:42 -0800 + Commit: 46bcb95, github.com/apache/spark/pull/4 + + Remove remaining references to incubation + Patrick Wendell + 2014-03-02 01:00:16 -0800 + Commit: 1fd2bfd, github.com/apache/spark/pull/51 + + Update io.netty from 4.0.13 Final to 4.0.17.Final + Binh Nguyen , Binh Nguyen + 2014-03-02 00:48:50 -0800 + Commit: b70823c, github.com/apache/spark/pull/41 + + Merge the old sbt-launch-lib.bash with the new sbt-launcher jar downloading logic. + Michael Armbrust + 2014-03-02 00:35:23 -0800 + Commit: 012bd5f, github.com/apache/spark/pull/14 + + Initialized the regVal for first iteration in SGD optimizer + DB Tsai + 2014-03-02 00:31:59 -0800 + Commit: 6fc76e4, github.com/apache/spark/pull/40 + + [SPARK-1100] prevent Spark from overwriting directory silently + CodingCat + 2014-03-01 17:27:54 -0800 + Commit: 3a8b698, github.com/apache/spark/pull/11 + + [SPARK-1150] fix repo location in create script (re-open) + CodingCat + 2014-03-01 17:24:53 -0800 + Commit: fe195ae, github.com/apache/spark/pull/52 + + Revert "[SPARK-1150] fix repo location in create script" + Patrick Wendell + 2014-03-01 17:15:38 -0800 + Commit: ec992e1 + + [SPARK-1150] fix repo location in create script + Mark Grover + 2014-03-01 16:21:22 -0800 + Commit: 9aa0957, github.com/apache/spark/pull/48 + + [SPARK-979] Randomize order of offers. + Kay Ousterhout + 2014-03-01 11:24:22 -0800 + Commit: 556c566, github.com/apache/spark/pull/27 + + SPARK-1151: Update dev merge script to use spark.git instead of incubator-spark + Thomas Graves + 2014-02-28 18:28:33 -0800 + Commit: 4ba3f70, github.com/apache/spark/pull/47 + + SPARK-1051. On YARN, executors don't doAs submitting user + Sandy Ryza + 2014-02-28 12:43:01 -0600 + Commit: 46dff34, github.com/apache/spark/pull/29 + + SPARK-1032. If Yarn app fails before registering, app master stays aroun... + Sandy Ryza + 2014-02-28 09:40:47 -0600 + Commit: 5f419bf, github.com/apache/spark/pull/28 + + Remote BlockFetchTracker trait + Kay Ousterhout + 2014-02-27 21:52:55 -0800 + Commit: edf8a56, github.com/apache/spark/pull/39 + + Removed reference to incubation in Spark user docs. + Reynold Xin + 2014-02-27 21:13:22 -0800 + Commit: 40e080a, github.com/apache/spark/pull/2 + + [HOTFIX] Patching maven build after #6 (SPARK-1121). + Patrick Wendell + 2014-02-27 15:06:20 -0800 + Commit: c42557b, github.com/apache/spark/pull/37 + + SPARK 1084.1 (resubmitted) + Sean Owen + 2014-02-27 11:12:21 -0800 + Commit: 12bbca2, github.com/apache/spark/pull/31 + + Show Master status on UI page + Raymond Liu + 2014-02-26 23:51:32 -0800 + Commit: aace2c0, github.com/apache/spark/pull/24 + + [SPARK-1089] fix the regression problem on ADD_JARS in 0.9 + CodingCat + 2014-02-26 23:42:15 -0800 + Commit: 345df5f, github.com/apache/spark/pull/13 + + SPARK-1121 Only add avro if the build is for Hadoop 0.23.X and SPARK_YARN is set + Prashant Sharma + 2014-02-26 23:40:49 -0800 + Commit: 6ccd6c5, github.com/apache/spark/pull/6 + + SPARK-1129: use a predefined seed when seed is zero in XORShiftRandom + Xiangrui Meng + 2014-02-26 23:22:30 -0800 + Commit: 5a3ad10, github.com/apache/spark/pull/645 + + Remove references to ClusterScheduler (SPARK-1140) + Kay Ousterhout + 2014-02-26 22:52:42 -0800 + Commit: 71f69d6, github.com/apache/spark/pull/9 + + Updated link for pyspark examples in docs + Jyotiska NK + 2014-02-26 21:37:04 -0800 + Commit: 2645035, github.com/apache/spark/pull/22 + + Deprecated and added a few java api methods for corresponding scala api. + Prashant Sharma + 2014-02-26 21:17:44 -0800 + Commit: 0e40e2b, github.com/apache/spark/pull/19 + + Removed reference to incubation in README.md. + Reynold Xin + 2014-02-26 16:52:26 -0800 + Commit: 84f7ca1, github.com/apache/spark/pull/1 + + SPARK-1115: Catch depickling errors + Bouke van der Bijl + 2014-02-26 14:50:37 -0800 + Commit: 12738c1, github.com/apache/spark/pull/644 + + SPARK-1135: fix broken anchors in docs + Matei Zaharia + 2014-02-26 11:20:16 -0800 + Commit: c86eec5, github.com/apache/spark/pull/3 + + SPARK-1078: Replace lift-json with json4s-jackson. + William Benton + 2014-02-26 10:09:50 -0800 + Commit: fbedc8e, github.com/apache/spark/pull/582 + + SPARK-1053. Don't require SPARK_YARN_APP_JAR + Sandy Ryza + 2014-02-26 10:00:02 -0600 + Commit: b8a1871, github.com/apache/spark/pull/553 + + For SPARK-1082, Use Curator for ZK interaction in standalone cluster + Raymond Liu + 2014-02-24 23:20:38 -0800 + Commit: c852201, github.com/apache/incubator-spark/pull/611 + + Graph primitives2 + Semih Salihoglu + 2014-02-24 22:42:30 -0800 + Commit: 1f4c7f7, github.com/apache/incubator-spark/pull/580 + + Include reference to twitter/chill in tuning docs + Andrew Ash + 2014-02-24 21:13:38 -0800 + Commit: a4f4fbc, github.com/apache/incubator-spark/pull/647 + + For outputformats that are Configurable, call setConf before sending data to them. + Bryn Keller + 2014-02-24 17:35:22 -0800 + Commit: 4d88030, github.com/apache/incubator-spark/pull/638 + + d8d190e 2014-02-24 16:58:57 -0800 + Merge pull request #641 from mateiz/spark-1124-master + [SPARK-1124: Fix infinite retries of reduce stage when a map stage failed] + + Fix removal from shuffleToMapStage to search for a key-value pair with our stage instead of using our shuffleID. + Matei Zaharia + 2014-02-24 13:14:56 -0800 + Commit: 0187cef + + SPARK-1124: Fix infinite retries of reduce stage when a map stage failed + Matei Zaharia + 2014-02-23 23:45:48 -0800 + Commit: cd32d5e + + SPARK-1071: Tidy logging strategy and use of log4j + Sean Owen + 2014-02-23 11:40:55 -0800 + Commit: c0ef3af, github.com/apache/incubator-spark/pull/570 + + [SPARK-1041] remove dead code in start script, remind user to set that in spark-env.sh + CodingCat + 2014-02-22 20:21:15 -0800 + Commit: 437b62f, github.com/apache/incubator-spark/pull/588 + + Migrate Java code to Scala or move it to src/main/java + Punya Biswal + 2014-02-22 17:53:48 -0800 + Commit: 29ac7ea, github.com/apache/incubator-spark/pull/605 + + [SPARK-1055] fix the SCALA_VERSION and SPARK_VERSION in docker file + CodingCat , Nan Zhu + 2014-02-22 15:39:25 -0800 + Commit: 1aa4f8a, github.com/apache/incubator-spark/pull/634 + + doctest updated for mapValues, flatMapValues in rdd.py + jyotiska + 2014-02-22 15:10:31 -0800 + Commit: 722199f, github.com/apache/incubator-spark/pull/621 + + Fixed minor typo in worker.py + jyotiska + 2014-02-22 10:09:50 -0800 + Commit: 3ff077d, github.com/apache/incubator-spark/pull/630 + + SPARK-1117: update accumulator docs + Xiangrui Meng + 2014-02-21 22:44:45 -0800 + Commit: aaec7d4, github.com/apache/incubator-spark/pull/631 + + [SPARK-1113] External spilling - fix Int.MaxValue hash code collision bug + Andrew Or + 2014-02-21 20:05:39 -0800 + Commit: fefd22f, github.com/apache/incubator-spark/pull/624 + + MLLIB-25: Implicit ALS runs out of memory for moderately large numbers of features + Sean Owen + 2014-02-21 12:46:12 -0800 + Commit: c8a4c9b, github.com/apache/incubator-spark/pull/629 + + SPARK-1111: URL Validation Throws Error for HDFS URL's + Patrick Wendell + 2014-02-21 11:11:55 -0800 + Commit: 45b15e2, github.com/apache/incubator-spark/pull/625 + + SPARK-1114: Allow PySpark to use existing JVM and Gateway + Ahir Reddy + 2014-02-20 21:20:39 -0800 + Commit: 59b1379, github.com/apache/incubator-spark/pull/622 + + Super minor: Add require for mergeCombiners in combineByKey + Aaron Davidson + 2014-02-20 16:46:13 -0800 + Commit: 3fede48, github.com/apache/incubator-spark/pull/623 + + MLLIB-22. Support negative implicit input in ALS + Sean Owen + 2014-02-19 23:44:53 -0800 + Commit: 9e63f80, github.com/apache/incubator-spark/pull/500 + + MLLIB-24: url of "Collaborative Filtering for Implicit Feedback Datasets" in ALS is invalid now + Chen Chao + 2014-02-19 22:06:35 -0800 + Commit: f9b7d64, github.com/apache/incubator-spark/pull/619 + + [SPARK-1105] fix site scala version error in docs + CodingCat + 2014-02-19 15:54:03 -0800 + Commit: 7b012c9, github.com/apache/incubator-spark/pull/618 + + SPARK-1106: check key name and identity file before launch a cluster + Xiangrui Meng + 2014-02-18 18:30:02 -0800 + Commit: b61435c, github.com/apache/incubator-spark/pull/617 + + Revert "[SPARK-1105] fix site scala version error in doc" + Patrick Wendell + 2014-02-18 17:46:47 -0800 + Commit: d9bb32a + + [SPARK-1105] fix site scala version error in doc + CodingCat + 2014-02-18 16:29:23 -0800 + Commit: d99773d, github.com/apache/incubator-spark/pull/616 + + Optimized imports + NirmalReddy , NirmalReddy + 2014-02-18 14:44:36 -0800 + Commit: ccb327a, github.com/apache/incubator-spark/pull/613 + + SPARK-1098: Minor cleanup of ClassTag usage in Java API + Aaron Davidson + 2014-02-17 19:23:27 -0800 + Commit: f74ae0e, github.com/apache/incubator-spark/pull/604 + + [SPARK-1090] improvement on spark_shell (help information, configure memory) + CodingCat + 2014-02-17 15:12:52 -0800 + Commit: e0d49ad, github.com/apache/incubator-spark/pull/599 + + Fix typos in Spark Streaming programming guide + Andrew Or + 2014-02-17 10:59:02 -0800 + Commit: 767e3ae, github.com/apache/incubator-spark/pull/536 + + Worker registration logging fix + Andrew Ash + 2014-02-17 09:51:55 -0800 + Commit: c0795cf, github.com/apache/incubator-spark/pull/608 + + Add subtractByKey to the JavaPairRDD wrapper + Punya Biswal + 2014-02-16 18:55:59 -0800 + Commit: 5af4477, github.com/apache/incubator-spark/pull/600 + + fix for https://spark-project.atlassian.net/browse/SPARK-1052 + Bijay Bisht + 2014-02-16 16:52:57 -0800 + Commit: 73cfdcf, github.com/apache/incubator-spark/pull/568 + + [SPARK-1092] print warning information if user use SPARK_MEM to regulate executor memory usage + CodingCat + 2014-02-16 12:25:38 -0800 + Commit: 1cad381, github.com/apache/incubator-spark/pull/602 + + Typo: Standlone -> Standalone + Andrew Ash + 2014-02-14 10:01:01 -0800 + Commit: eec4bd1, github.com/apache/incubator-spark/pull/601 + + 2414ed3 2014-02-13 14:26:06 -0800 + Merge pull request #598 from shivaram/master. + [Update spark_ec2 to use 0.9.0 by default] + + Add c3 instance types to Spark EC2 + Christian Lundgren + 2014-02-13 12:44:21 -0800 + Commit: 5fa53c0, github.com/apache/incubator-spark/pull/595 + + Ported hadoopClient jar for < 1.0.1 fix + Bijay Bisht + 2014-02-12 23:42:10 -0800 + Commit: a3bb861, github.com/apache/incubator-spark/pull/584 + + SPARK-1073 Keep GitHub pull request title as commit summary + Andrew Ash + 2014-02-12 23:23:06 -0800 + Commit: 6ee0ad8, github.com/apache/incubator-spark/pull/574 + + 7fe7a55 2014-02-12 22:35:09 -0800 + Merge pull request #592 from rxin/test. + [SPARK-1088: Create a script for running tests so we can have version specific testing on Jenkins.] + + 7e29e02 2014-02-12 16:26:25 -0800 + Merge pull request #591 from mengxr/transient-new. + [SPARK-1076: [Fix #578] add @transient to some vals] + + 2bea070 2014-02-12 10:47:52 -0800 + Merge pull request #589 from mengxr/index. + [SPARK-1076: Convert Int to Long to avoid overflow] + + e733d65 2014-02-12 00:42:42 -0800 + Merge pull request #578 from mengxr/rank. + [SPARK-1076: zipWithIndex and zipWithUniqueId to RDD] + + 68b2c0d 2014-02-11 22:39:48 -0800 + Merge pull request #583 from colorant/zookeeper. + [Minor fix for ZooKeeperPersistenceEngine to use configured working dir] + + b0dab1b 2014-02-11 14:48:59 -0800 + Merge pull request #571 from holdenk/switchtobinarysearch. + [SPARK-1072 Use binary search when needed in RangePartioner] + + ba38d98 2014-02-11 14:46:22 -0800 + Merge pull request #577 from hsaputra/fix_simple_streaming_doc. + [SPARK-1075 Fix doc in the Spark Streaming custom receiver closing bracket in the class constructor] + + 4afe6cc 2014-02-10 22:28:39 -0800 + Merge pull request #579 from CrazyJvm/patch-1. + ["in the source DStream" rather than "int the source DStream"] + + d6a9bdc 2014-02-09 23:35:06 -0800 + Revert "Merge pull request #560 from pwendell/logging. Closes #560." + [This reverts commit b6d40b782327188a25ded5b22790552121e5271f.] + + 919bd7f 2014-02-09 22:17:52 -0800 + Merge pull request #567 from ScrapCodes/style2. + [SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build. Pt 2] + + 2182aa3 2014-02-09 15:19:50 -0800 + Merge pull request #566 from martinjaggi/copy-MLlib-d. + [new MLlib documentation for optimization, regression and classification] + + afc8f3c 2014-02-09 13:57:29 -0800 + Merge pull request #551 from qqsun8819/json-protocol. + [[SPARK-1038] Add more fields in JsonProtocol and add tests that verify the JSON itself] + + 94ccf86 2014-02-09 13:54:27 -0800 + Merge pull request #569 from pwendell/merge-fixes. + [Fixes bug where merges won't close associated pull request.] + + b69f8b2 2014-02-09 10:09:19 -0800 + Merge pull request #557 from ScrapCodes/style. Closes #557. + [SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build.] + + b6dba10 2014-02-08 23:39:17 -0800 + Merge pull request #556 from CodingCat/JettyUtil. Closes #556. + [[SPARK-1060] startJettyServer should explicitly use IP information] + + 2ef37c9 2014-02-08 23:36:48 -0800 + Merge pull request #562 from jyotiska/master. Closes #562. + [Added example Python code for sort] + + b6d40b7 2014-02-08 23:35:31 -0800 + Merge pull request #560 from pwendell/logging. Closes #560. + [[WIP] SPARK-1067: Default log4j initialization causes errors for those not using log4j] + + f892da8 2014-02-08 23:13:34 -0800 + Merge pull request #565 from pwendell/dev-scripts. Closes #565. + [SPARK-1066: Add developer scripts to repository.] + + c2341c9 2014-02-08 16:00:43 -0800 + Merge pull request #542 from markhamstra/versionBump. Closes #542. + [Version number to 1.0.0-SNAPSHOT] + + f0ce736 2014-02-08 12:59:48 -0800 + Merge pull request #561 from Qiuzhuang/master. Closes #561. + [Kill drivers in postStop() for Worker.] + + 7805080 2014-02-08 12:24:08 -0800 + Merge pull request #454 from jey/atomic-sbt-download. Closes #454. + [Make sbt download an atomic operation] + + fabf174 2014-02-08 11:39:13 -0800 + Merge pull request #552 from martinjaggi/master. Closes #552. + [tex formulas in the documentation] + + 3a9d82c 2014-02-06 22:38:36 -0800 + Merge pull request #506 from ash211/intersection. Closes #506. + [SPARK-1062 Add rdd.intersection(otherRdd) method] + + 1896c6e 2014-02-06 22:05:53 -0800 + Merge pull request #533 from andrewor14/master. Closes #533. + [External spilling - generalize batching logic] + + 0b448df 2014-02-06 16:15:24 -0800 + Merge pull request #450 from kayousterhout/fetch_failures. Closes #450. + [Only run ResubmitFailedStages event after a fetch fails] + + 18ad59e 2014-02-06 16:10:48 -0800 + Merge pull request #321 from kayousterhout/ui_kill_fix. Closes #321. + [Inform DAG scheduler about all started/finished tasks.] + + 446403b 2014-02-06 15:41:16 -0800 + Merge pull request #554 from sryza/sandy-spark-1056. Closes #554. + [SPARK-1056. Fix header comment in Executor to not imply that it's only u...] + + 084839b 2014-02-06 14:58:35 -0800 + Merge pull request #498 from ScrapCodes/python-api. Closes #498. + [Python api additions] + + 79c9552 2014-02-05 23:38:12 -0800 + Merge pull request #545 from kayousterhout/fix_progress. Closes #545. + [Fix off-by-one error with task progress info log.] + + 3802096 2014-02-05 23:37:07 -0800 + Merge pull request #526 from tgravescs/yarn_client_stop_am_fix. Closes #526. + [spark on yarn - yarn-client mode doesn't always exit immediately] + + 18c4ee7 2014-02-05 22:08:47 -0800 + Merge pull request #549 from CodingCat/deadcode_master. Closes #549. + [remove actorToWorker in master.scala, which is actually not used] + + cc14ba9 2014-02-05 12:44:24 -0800 + Merge pull request #544 from kayousterhout/fix_test_warnings. Closes #544. + [Fixed warnings in test compilation.] + + f7fd80d 2014-02-05 10:29:45 -0800 + Merge pull request #540 from sslavic/patch-3. Closes #540. + [Fix line end character stripping for Windows] + + 9209287 2014-02-04 09:47:11 -0800 + Merge pull request #534 from sslavic/patch-1. Closes #534. + [Fixed wrong path to compute-classpath.cmd] + + 0c05cd3 2014-02-04 09:45:46 -0800 + Merge pull request #535 from sslavic/patch-2. Closes #535. + [Fixed typo in scaladoc] + + 23af00f 2014-02-03 13:02:09 -0800 + Merge pull request #528 from mengxr/sample. Closes #528. + [ Refactor RDD sampling and add randomSplit to RDD (update)] + + 1625d8c 2014-02-03 11:25:39 -0800 + Merge pull request #530 from aarondav/cleanup. Closes #530. + [Remove explicit conversion to PairRDDFunctions in cogroup()] + + 0386f42 2014-02-02 21:51:17 -0800 + Merge pull request #529 from hsaputra/cleanup_right_arrowop_scala + [Change the ⇒ character (maybe from scalariform) to => in Scala code for style consistency] + + a8cf3ec 2014-01-31 16:52:02 -0800 + Merge pull request #527 from ankurdave/graphx-assembly-pom + [Add GraphX to assembly/pom.xml] + + ac712e4 2014-01-30 09:33:18 -0800 + Merge pull request #524 from rxin/doc + [Added spark.shuffle.file.buffer.kb to configuration doc.] + + 0ff38c2 2014-01-29 12:44:54 -0800 + Merge pull request #494 from tyro89/worker_registration_issue + [Issue with failed worker registrations] + + 7930209 2014-01-28 21:51:05 -0800 + Merge pull request #497 from tdas/docs-update + [Updated Spark Streaming Programming Guide] + + f8c742c 2014-01-28 21:30:20 -0800 + Merge pull request #523 from JoshRosen/SPARK-1043 + [Switch from MUTF8 to UTF8 in PySpark serializers.] + + Switch from MUTF8 to UTF8 in PySpark serializers. + Josh Rosen + 2014-01-28 19:50:26 -0800 + Commit: 1381fc7 + + 84670f2 2014-01-27 17:08:35 -0800 + Merge pull request #466 from liyinan926/file-overwrite-new + [Allow files added through SparkContext.addFile() to be overwritten] + + 3d5c03e 2014-01-27 16:27:01 -0800 + Merge pull request #516 from sarutak/master + [modified SparkPluginBuild.scala to use https protocol for accessing gith...] + + f16c21e 2014-01-27 14:24:06 -0800 + Merge pull request #490 from hsaputra/modify_checkoption_with_isdefined + [Replace the check for None Option with isDefined and isEmpty in Scala code] + + f67ce3e 2014-01-27 11:15:51 -0800 + Merge pull request #460 from srowen/RandomInitialALSVectors + [Choose initial user/item vectors uniformly on the unit sphere] + + modified SparkPluginBuild.scala to use https protocol for accessing github. + sarutak + 2014-01-27 17:00:26 +0900 + Commit: 6a5af7b + + c40619d 2014-01-25 22:41:30 -0800 + Merge pull request #504 from JoshRosen/SPARK-1025 + [Fix PySpark hang when input files are deleted (SPARK-1025)] + + c66a2ef 2014-01-25 22:36:07 -0800 + Merge pull request #511 from JoshRosen/SPARK-1040 + [Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)] + + Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040) + Josh Rosen + 2014-01-25 16:39:20 -0800 + Commit: 740e865 + + Increase JUnit test verbosity under SBT. + Josh Rosen + 2014-01-25 16:32:44 -0800 + Commit: 531d9d7 + + 05be704 2014-01-23 20:53:18 -0800 + Merge pull request #505 from JoshRosen/SPARK-1026 + [Deprecate mapPartitionsWithSplit in PySpark (SPARK-1026)] + + Deprecate mapPartitionsWithSplit in PySpark. + Josh Rosen + 2014-01-23 20:01:36 -0800 + Commit: 4cebb79 + + 3d6e754 2014-01-23 19:47:00 -0800 + Merge pull request #503 from pwendell/master + [Fix bug on read-side of external sort when using Snappy.] + + Minor fix + Patrick Wendell + 2014-01-23 19:23:12 -0800 + Commit: ff44732 + + c319617 2014-01-23 19:11:59 -0800 + Merge pull request #502 from pwendell/clone-1 + [Remove Hadoop object cloning and warn users making Hadoop RDD's.] + + cad3002 2014-01-23 19:08:34 -0800 + Merge pull request #501 from JoshRosen/cartesian-rdd-fixes + [Fix two bugs in PySpark cartesian(): SPARK-978 and SPARK-1034] + + Minor changes after auditing diff from earlier version + Patrick Wendell + 2014-01-23 18:30:11 -0800 + Commit: 268ecbd + + Fix for SPARK-1025: PySpark hang on missing files. + Josh Rosen + 2014-01-23 18:10:16 -0800 + Commit: f830684 + + Response to Matei's review + Patrick Wendell + 2014-01-23 18:12:40 -0800 + Commit: c58d4ea + + Fix bug on read-side of external sort when using Snappy. + Patrick Wendell + 2014-01-23 17:59:42 -0800 + Commit: 0213b40 + + Remove Hadoop object cloning and warn users making Hadoop RDD's. + Patrick Wendell + 2014-01-23 13:30:54 -0800 + Commit: 7101017 + + Fix SPARK-978: ClassCastException in PySpark cartesian. + Josh Rosen + 2014-01-23 15:09:19 -0800 + Commit: 6156990 + + Fix SPARK-1034: Py4JException on PySpark Cartesian Result + Josh Rosen + 2014-01-23 13:05:59 -0800 + Commit: 0035dbb + + fad6aac 2014-01-23 11:14:15 -0800 + Merge pull request #406 from eklavya/master + [Extending Java API coverage] + + a2b47da 2014-01-23 10:48:26 -0800 + Merge pull request #499 from jianpingjwang/dev1 + [Replace commons-math with jblas in SVDPlusPlus] + + fixed ClassTag in mapPartitions + eklavya + 2014-01-23 17:40:36 +0530 + Commit: 60e7457 + + Add jblas dependency + Jianping J Wang + 2014-01-23 19:54:01 +0800 + Commit: 19a01c1 + + Add jblas dependency + Jianping J Wang + 2014-01-23 19:48:39 +0800 + Commit: a5a513e + + Replace commons-math with jblas + Jianping J Wang + 2014-01-23 19:44:30 +0800 + Commit: cc0fd33 + + a1cd185 2014-01-22 19:37:29 -0800 + Merge pull request #496 from pwendell/master + [Fix bug in worker clean-up in UI] + + 034dce2 2014-01-22 18:58:02 -0800 + Merge pull request #447 from CodingCat/SPARK-1027 + [fix for SPARK-1027] + + Fix bug in worker clean-up in UI + Patrick Wendell + 2014-01-22 18:19:46 -0800 + Commit: 6285513 + + refactor sparkHome to val + CodingCat + 2014-01-22 19:32:51 -0500 + Commit: 2b3c461 + + 3184fac 2014-01-22 15:45:04 -0800 + Merge pull request #495 from srowen/GraphXCommonsMathDependency + [Fix graphx Commons Math dependency] + + Also add graphx commons-math3 dependeny in sbt build + Sean Owen + 2014-01-22 22:40:41 +0000 + Commit: 4476398 + + a1238bb 2014-01-22 14:32:59 -0800 + Merge pull request #492 from skicavs/master + [fixed job name and usage information for the JavaSparkPi example] + + Depend on Commons Math explicitly instead of accidentally getting it from Hadoop (which stops working in 2.2.x) and also use the newer commons-math3 + Sean Owen + 2014-01-22 22:25:49 +0000 + Commit: fd0c5b8 + + 576c4a4 2014-01-22 14:10:07 -0800 + Merge pull request #478 from sryza/sandy-spark-1033 + [SPARK-1033. Ask for cores in Yarn container requests] + + 5bcfd79 2014-01-22 14:05:48 -0800 + Merge pull request #493 from kayousterhout/double_add + [Fixed bug where task set managers are added to queue twice] + + d009b17 2014-01-22 14:01:30 -0800 + Merge pull request #315 from rezazadeh/sparsesvd + [Sparse SVD] + + Fixed bug where task set managers are added to queue twice + Kay Ousterhout + 2014-01-22 09:49:31 -0800 + Commit: 19da82c + + fixed job name and usage information for the JavaSparkPi example + Kevin Mader + 2014-01-22 15:58:23 +0100 + Commit: 36f9a64 + + Replace the code to check for Option != None with Option.isDefined call in Scala code. + Henry Saputra + 2014-01-21 23:22:10 -0800 + Commit: 90ea9d5 + + 749f842 2014-01-21 14:53:49 -0800 + Merge pull request #489 from ash211/patch-6 + [Clarify spark.default.parallelism] + + Clarify spark.default.parallelism + Andrew Ash + 2014-01-21 14:49:35 -0800 + Commit: 069bb94 + + f854498 2014-01-21 10:49:54 -0800 + Merge pull request #469 from ajtulloch/use-local-spark-context-in-tests-for-mllib + [[MLlib] Use a LocalSparkContext trait in test suites] + + Fixed import order + Andrew Tulloch + 2014-01-21 13:34:59 +0000 + Commit: 3a067b4 + + Incorporate Tom's comments - update doc and code to reflect that core requests may not always be honored + Sandy Ryza + 2014-01-21 00:38:02 -0800 + Commit: adf4261 + + 77b986f 2014-01-21 00:09:42 -0800 + Merge pull request #480 from pwendell/0.9-fixes + [Handful of 0.9 fixes] + + Style clean-up + Patrick Wendell + 2014-01-20 23:42:24 -0800 + Commit: a9bcc98 + + c67d3d8 2014-01-20 23:34:35 -0800 + Merge pull request #484 from tdas/run-example-fix + [Made run-example respect SPARK_JAVA_OPTS and SPARK_MEM.] + + Removed SPARK_MEM from run-examples. + Tathagata Das + 2014-01-20 23:15:28 -0800 + Commit: 65869f8 + + Adding small code comment + Patrick Wendell + 2014-01-20 23:11:45 -0800 + Commit: a917a87 + + 6b4eed7 2014-01-20 22:35:45 -0800 + Merge pull request #449 from CrazyJvm/master + [SPARK-1028 : fix "set MASTER automatically fails" bug.] + + 0367981 2014-01-20 22:25:50 -0800 + Merge pull request #482 from tdas/streaming-example-fix + [Added StreamingContext.awaitTermination to streaming examples] + + 7373ffb 2014-01-20 21:44:29 -0800 + Merge pull request #483 from pwendell/gitignore + [Restricting /lib to top level directory in .gitignore] + + Made run-example respect SPARK_JAVA_OPTS and SPARK_MEM. + Tathagata Das + 2014-01-20 20:48:59 -0800 + Commit: e0b741d + + Restricting /lib to top level directory in .gitignore + Patrick Wendell + 2014-01-20 20:39:10 -0800 + Commit: e437069 + + Added StreamingContext.awaitTermination to streaming examples. + Tathagata Das + 2014-01-20 20:25:04 -0800 + Commit: 2e95174 + + Avoid matching attempt files in the checkpoint + Patrick Wendell + 2014-01-20 20:02:02 -0800 + Commit: d46df96 + + Remove shuffle files if they are still present on a machine. + Patrick Wendell + 2014-01-20 19:11:22 -0800 + Commit: de526ad + + Fixing speculation bug + Patrick Wendell + 2014-01-20 19:05:03 -0800 + Commit: f84400e + + Force use of LZF when spilling data + Patrick Wendell + 2014-01-20 19:00:48 -0800 + Commit: c324ac1 + + Bug fix for reporting of spill output + Patrick Wendell + 2014-01-20 18:34:00 -0800 + Commit: 1b29914 + + Minor fixes + Patrick Wendell + 2014-01-20 18:33:21 -0800 + Commit: 54867e9 + + Removing docs on akka options + Patrick Wendell + 2014-01-20 16:35:26 -0800 + Commit: cdb003e + + SPARK-1033. Ask for cores in Yarn container requests + Sandy Ryza + 2014-01-19 10:16:25 -0800 + Commit: 3e85b87 + + fix for SPARK-1027 + CodingCat + 2014-01-15 20:46:14 -0500 + Commit: 29f4b6a + + executor creation failed should not make the worker restart + CodingCat + 2014-01-15 19:32:50 -0500 + Commit: f9a95d6 + + 792d908 2014-01-19 11:33:11 -0800 + Merge pull request #470 from tgravescs/fix_spark_examples_yarn + [Only log error on missing jar to allow spark examples to jar.] + + 256a355 2014-01-19 10:29:54 -0800 + Merge pull request #458 from tdas/docs-update + [Updated java API docs for streaming, along with very minor changes in the code examples.] + + update comment + Thomas Graves + 2014-01-19 12:21:39 -0600 + Commit: dd56b21 + + Only log error on missing jar to allow spark examples to jar. + Thomas Graves + 2014-01-19 12:16:58 -0600 + Commit: ceb79a3 + + LocalSparkContext for MLlib + Andrew Tulloch + 2014-01-19 17:51:00 +0000 + Commit: 720836a + + Addressed comments from Reynold + Yinan Li + 2014-01-18 21:28:17 -0800 + Commit: 584323c + + fe8a354 2014-01-18 16:29:23 -0800 + Merge pull request #459 from srowen/UpdaterL2Regularization + [Correct L2 regularized weight update with canonical form] + + 73dfd42 2014-01-18 16:23:56 -0800 + Merge pull request #437 from mridulm/master + [Minor api usability changes] + + 4c16f79 2014-01-18 16:21:43 -0800 + Merge pull request #426 from mateiz/py-ml-tests + [Re-enable Python MLlib tests (require Python 2.7 and NumPy 1.7+)] + + bf56995 2014-01-18 16:17:34 -0800 + Merge pull request #462 from mateiz/conf-file-fix + [Remove Typesafe Config usage and conf files to fix nested property names] + + Allow files added through SparkContext.addFile() to be overwritten + Yinan Li + 2014-01-17 17:27:25 -0800 + Commit: fd833e7 + + aa981e4 2014-01-18 12:49:21 -0800 + Merge pull request #461 from pwendell/master + [Use renamed shuffle spill config in CoGroupedRDD.scala] + + Use renamed shuffle spill config in CoGroupedRDD.scala + Patrick Wendell + 2014-01-18 11:55:10 -0800 + Commit: 5316bca + + Correct L2 regularized weight update with canonical form + Sean Owen + 2014-01-18 12:53:01 +0000 + Commit: e91ad3f + + rename to MatrixSVD + Reza Zadeh + 2014-01-17 14:40:51 -0800 + Commit: 85b95d0 + + rename to MatrixSVD + Reza Zadeh + 2014-01-17 14:39:30 -0800 + Commit: fa32998 + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-17 14:34:03 -0800 + Commit: caf97a2 + + make example 0-indexed + Reza Zadeh + 2014-01-17 14:33:03 -0800 + Commit: 4e96757 + + 0index docs + Reza Zadeh + 2014-01-17 14:31:39 -0800 + Commit: 5c639d7 + + prettify + Reza Zadeh + 2014-01-17 14:14:29 -0800 + Commit: c9b4845 + + add rename computeSVD + Reza Zadeh + 2014-01-17 13:59:05 -0800 + Commit: dbec69b + + replace this.type with SVD + Reza Zadeh + 2014-01-17 13:57:27 -0800 + Commit: eb2d8c4 + + use 0-indexing + Reza Zadeh + 2014-01-17 13:55:42 -0800 + Commit: cb13b15 + + changes from PR + Reza Zadeh + 2014-01-17 13:39:40 -0800 + Commit: d28bf41 + + Address review comment + Mridul Muralidharan + 2014-01-17 18:28:55 +0530 + Commit: b690e11 + + d749d47 2014-01-16 23:18:15 -0800 + Merge pull request #451 from Qiuzhuang/master + [Fixed Window spark shell launch script error.] + + d4fd89e 2014-01-16 23:17:30 -0800 + Merge pull request #438 from ScrapCodes/clone-records-java-api + [Clone records java api] + + adding clone records field to equivaled java apis + Prashant Sharma + 2014-01-14 20:13:55 +0530 + Commit: fcb4fc6 + + Updated java API docs for streaming, along with very minor changes in the code examples. + Tathagata Das + 2014-01-16 14:44:02 -0800 + Commit: 11e6534 + + Use method, not variable + Mridul Muralidharan + 2014-01-16 17:26:42 +0530 + Commit: edd82c5 + + Address review comments + Mridul Muralidharan + 2014-01-16 17:23:25 +0530 + Commit: 1a0da89 + + Fixed Window spark shell launch script error. JIRA SPARK-1029:https://spark-project.atlassian.net/browse/SPARK-1029 + Qiuzhuang Lian + 2014-01-16 16:09:10 +0800 + Commit: 4e510b0 + + c06a307 2014-01-15 23:47:25 -0800 + Merge pull request #445 from kayousterhout/exec_lost + [Fail rather than hanging if a task crashes the JVM.] + + Updated unit test comment + Kay Ousterhout + 2014-01-15 23:46:14 -0800 + Commit: 718a13c + + 84595ea 2014-01-15 20:15:29 -0800 + Merge pull request #414 from soulmachine/code-style + [Code clean up for mllib] + + fix some format problem. + CrazyJvm + 2014-01-16 11:57:46 +0800 + Commit: 8400536 + + fix "set MASTER automatically fails" bug. + CrazyJvm + 2014-01-16 11:45:02 +0800 + Commit: 7a0c5b5 + + 0675ca5 2014-01-15 16:09:03 -0800 + Merge pull request #439 from CrazyJvm/master + [SPARK-1024 Remove "-XX:+UseCompressedStrings" option from tuning guide] + + Fail rather than hanging if a task crashes the JVM. + Kay Ousterhout + 2014-01-15 16:03:40 -0800 + Commit: a268d63 + + 4f0c361 2014-01-15 14:25:45 -0800 + Merge pull request #444 from mateiz/py-version + [Clarify that Python 2.7 is only needed for MLlib] + + Clarify that Python 2.7 is only needed for MLlib + Matei Zaharia + 2014-01-15 14:20:39 -0800 + Commit: 2ffdaef + + 59f475c 2014-01-15 13:55:14 -0800 + Merge pull request #442 from pwendell/standalone + [Workers should use working directory as spark home if it's not specified] + + 2a05403 2014-01-15 13:54:45 -0800 + Merge pull request #443 from tdas/filestream-fix + [Made some classes private[stremaing] and deprecated a method in JavaStreamingContext.] + + Made some classes private[stremaing] and deprecated a method in JavaStreamingContext. + Tathagata Das + 2014-01-15 12:15:46 -0800 + Commit: 9e63753 + + 5fecd25 2014-01-15 11:15:07 -0800 + Merge pull request #441 from pwendell/graphx-build + [GraphX shouldn't list Spark as provided.] + + Workers should use working directory as spark home if it's not specified + Patrick Wendell + 2014-01-15 10:58:02 -0800 + Commit: 00a3f7e + + GraphX shouldn't list Spark as provided + Patrick Wendell + 2014-01-15 10:44:17 -0800 + Commit: 9259d70 + + 494d3c0 2014-01-15 10:00:50 -0800 + Merge pull request #433 from markhamstra/debFix + [Updated Debian packaging] + + cef2af9 2014-01-15 10:06:17 -0600 + Merge pull request #366 from colorant/yarn-dev + [More yarn code refactor] + + remove "-XX:+UseCompressedStrings" option + CrazyJvm + 2014-01-15 22:26:15 +0800 + Commit: 263933d + + 3d9e66d 2014-01-14 23:17:05 -0800 + Merge pull request #436 from ankurdave/VertexId-case + [Rename VertexID -> VertexId in GraphX] + + Expose method and class - so that we can use it from user code (particularly since checkpoint directory is autogenerated now + Mridul Muralidharan + 2014-01-15 12:44:44 +0530 + Commit: 0aea33d + + 139c24e 2014-01-14 23:07:55 -0800 + Merge pull request #435 from tdas/filestream-fix + [Fixed the flaky tests by making SparkConf not serializable] + + 087487e 2014-01-14 22:50:36 -0800 + Merge pull request #434 from rxin/graphxmaven + [Fixed SVDPlusPlusSuite in Maven build.] + + Merge remote-tracking branch 'apache/master' into filestream-fix + Tathagata Das + 2014-01-14 22:21:20 -0800 + Commit: 0e15bd7 + + Changed SparkConf to not be serializable. And also fixed unit-test log paths in log4j.properties of external modules. + Tathagata Das + 2014-01-14 22:20:14 -0800 + Commit: 1f4718c + + Fixed SVDPlusPlusSuite in Maven build. + Reynold Xin + 2014-01-14 22:18:43 -0800 + Commit: dfb1524 + + Removed repl-bin and updated maven build doc. + Mark Hamstra + 2014-01-14 21:36:58 -0800 + Commit: 147a943 + + VertexID -> VertexId + Ankur Dave + 2014-01-14 22:17:18 -0800 + Commit: f4d9019 + + Add deb profile to assembly/pom.xml + Mark Hamstra + 2014-01-14 21:23:09 -0800 + Commit: 148757e + + 3a386e2 2014-01-14 21:52:50 -0800 + Merge pull request #424 from jegonzal/GraphXProgrammingGuide + [Additional edits for clarity in the graphx programming guide.] + + ad294db 2014-01-14 21:51:06 -0800 + Merge pull request #431 from ankurdave/graphx-caching-doc + [Describe caching and uncaching in GraphX programming guide] + + Describe GraphX caching and uncaching in guide + Ankur Dave + 2014-01-14 17:24:25 -0800 + Commit: 1210ec2 + + 74b46ac 2014-01-14 14:59:13 -0800 + Merge pull request #428 from pwendell/writeable-objects + [Don't clone records for text files] + + 193a075 2014-01-14 14:53:24 -0800 + Merge pull request #429 from ankurdave/graphx-examples-pom.xml + [Add GraphX dependency to examples/pom.xml] + + d601a76 2014-01-14 14:52:24 -0800 + Merge pull request #427 from pwendell/deprecate-aggregator + [Deprecate rather than remove old combineValuesByKey function] + + Add GraphX dependency to examples/pom.xml + Ankur Dave + 2014-01-14 13:57:51 -0800 + Commit: 8ea056d + + Style fix + Patrick Wendell + 2014-01-14 13:56:27 -0800 + Commit: b1b22b7 + + Adding fix covering combineCombinersByKey as well + Patrick Wendell + 2014-01-14 13:52:23 -0800 + Commit: 8ea2cd5 + + 2ce23a5 2014-01-14 13:28:44 -0800 + Merge pull request #425 from rxin/scaladoc + [API doc update & make Broadcast public] + + Complain if Python and NumPy versions are too old for MLlib + Matei Zaharia + 2014-01-14 12:27:58 -0800 + Commit: 5b3a3e2 + + Deprecate rather than remove old combineValuesByKey function + Patrick Wendell + 2014-01-14 12:15:10 -0800 + Commit: b683608 + + Re-enable Python MLlib tests (require Python 2.7 and NumPy 1.7+) + Matei Zaharia + 2014-01-14 12:14:48 -0800 + Commit: 938e4a0 + + Don't clone records for text files + Patrick Wendell + 2014-01-14 11:57:53 -0800 + Commit: 6f965a4 + + Fixed a typo in JavaSparkContext's API doc. + Reynold Xin + 2014-01-14 11:42:28 -0800 + Commit: f12e506 + + Maintain Serializable API compatibility by reverting back to java.io.Serializable for Broadcast and Accumulator. + Reynold Xin + 2014-01-14 11:30:59 -0800 + Commit: 1b5623f + + Added license header for package.scala in the Java API package. + Reynold Xin + 2014-01-14 11:20:12 -0800 + Commit: 55db774 + + Added package doc for the Java API. + Reynold Xin + 2014-01-14 11:16:25 -0800 + Commit: f8c12e9 + + Updated API doc for Accumulable and Accumulator. + Reynold Xin + 2014-01-14 11:16:08 -0800 + Commit: 6a12b9e + + Broadcast variable visibility change & doc update. + Reynold Xin + 2014-01-14 11:15:21 -0800 + Commit: 71b3007 + + Additional edits for clarity in the graphx programming guide. + Joseph E. Gonzalez + 2014-01-14 10:31:43 -0800 + Commit: 0bba773 + + 3fcc68b 2014-01-14 09:44:43 -0800 + Merge pull request #423 from jegonzal/GraphXProgrammingGuide + [Improving the graphx-programming-guide] + + Improving the graphx-programming-guide. + Joseph E. Gonzalez + 2014-01-14 09:40:06 -0800 + Commit: 486f37c + + Added parentheses for that getDouble() also has side effect + Frank Dai + 2014-01-14 18:56:11 +0800 + Commit: 57fcfc7 + + fa75e5e 2014-01-14 01:18:34 -0800 + Merge pull request #420 from pwendell/header-files + [Add missing header files] + + Add missing header files + Patrick Wendell + 2014-01-14 01:14:20 -0800 + Commit: 2303479 + + Modifications as suggested in PR feedback- + Saurabh Rawat + 2014-01-14 14:19:02 +0530 + Commit: 1442cd5 + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-13 23:52:34 -0800 + Commit: 845e568 + + Merge remote-tracking branch 'upstream/master' into code-style + Frank Dai + 2014-01-14 15:29:17 +0800 + Commit: a3da468 + + Indent two spaces + Frank Dai + 2014-01-14 14:59:01 +0800 + Commit: c2852cf + + Since getLong() and getInt() have side effect, get back parentheses, and remove an empty line + Frank Dai + 2014-01-14 14:53:10 +0800 + Commit: 12386b3 + + Code clean up for mllib + Frank Dai + 2014-01-14 14:37:26 +0800 + Commit: 0d94d74 + + Address comments to fix code formats + Raymond Liu + 2014-01-10 09:44:44 +0800 + Commit: 4c22c55 + + Yarn workerRunnable refactor + Raymond Liu + 2014-01-09 14:16:07 +0800 + Commit: 161ab93 + + Yarn Client refactor + Raymond Liu + 2014-01-09 09:53:50 +0800 + Commit: 79a5ba3 + + Modifications as suggested in PR feedback- + Saurabh Rawat + 2014-01-13 23:40:04 +0530 + Commit: e922973 + + Remove default param from mapPartitions + eklavya + 2014-01-13 18:13:22 +0530 + Commit: fa42951 + + Remove classtag from mapPartitions. + eklavya + 2014-01-13 18:09:58 +0530 + Commit: 8fe562c + + Added foreachPartition method to JavaRDD. + eklavya + 2014-01-13 17:56:47 +0530 + Commit: 6a65fee + + Added mapPartitions method to JavaRDD. + eklavya + 2014-01-13 17:56:10 +0530 + Commit: dbadc6b + + Added setter method setGenerator to JavaRDD. + eklavya + 2014-01-13 17:53:35 +0530 + Commit: aae8a01 + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-11 13:27:15 -0800 + Commit: f324d53 + + add dimension parameters to example + Reza Zadeh + 2014-01-10 21:30:54 -0800 + Commit: 1afdeae + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-09 22:45:32 -0800 + Commit: 21c8a54 + + fix example + Reza Zadeh + 2014-01-09 22:39:41 -0800 + Commit: cf5bd4a + + documentation for sparsematrix + Reza Zadeh + 2014-01-07 17:19:28 -0800 + Commit: 4f38b6f + + More sparse matrix usage. + Reza Zadeh + 2014-01-07 17:16:17 -0800 + Commit: 7d7490b + + fix docs to use SparseMatrix + Reza Zadeh + 2014-01-05 18:03:57 -0800 + Commit: 746148b + + use SparseMatrix everywhere + Reza Zadeh + 2014-01-04 14:28:07 -0800 + Commit: 06c0f76 + + prettify + Reza Zadeh + 2014-01-04 12:44:04 -0800 + Commit: cdff9fc + + new example file + Reza Zadeh + 2014-01-04 12:33:22 -0800 + Commit: e9bd6cb + + fix tests + Reza Zadeh + 2014-01-04 11:52:42 -0800 + Commit: 8bfcce1 + + set methods + Reza Zadeh + 2014-01-04 11:30:36 -0800 + Commit: 35adc72 + + add k parameter + Reza Zadeh + 2014-01-04 01:52:28 -0800 + Commit: 73daa70 + + using decomposed matrix struct now + Reza Zadeh + 2014-01-04 00:38:53 -0800 + Commit: 26a74f0 + + new return struct + Reza Zadeh + 2014-01-04 00:15:04 -0800 + Commit: d2d5e5e + + start using matrixentry + Reza Zadeh + 2014-01-03 22:17:24 -0800 + Commit: 7f631dd + + rename sparsesvd.scala + Reza Zadeh + 2014-01-03 21:55:38 -0800 + Commit: 6bcdb76 + + New matrix entry file + Reza Zadeh + 2014-01-03 21:54:57 -0800 + Commit: b059a2a + + fix error message + Reza Zadeh + 2014-01-02 01:51:38 -0800 + Commit: e617ae2 + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-02 01:50:30 -0800 + Commit: 6140578 + + more docs yay + Reza Zadeh + 2014-01-01 20:22:29 -0800 + Commit: 2612164 + + javadoc for sparsesvd + Reza Zadeh + 2014-01-01 20:20:16 -0800 + Commit: 915d53f + + old version of spark_ec2 + Reza Zadeh + 2014-01-01 20:08:01 -0800 + Commit: c868d71 + + remove accidental changes to ec2 script + Reza Zadeh + 2014-01-01 20:05:03 -0800 + Commit: 0c3797d + + doc tweaks + Reza Zadeh + 2014-01-01 20:03:47 -0800 + Commit: 53ccf65 + + doc tweak + Reza Zadeh + 2014-01-01 20:02:37 -0800 + Commit: 97dc527 + + doc tweaks + Reza Zadeh + 2014-01-01 20:01:13 -0800 + Commit: b941b6f + + tweaks to docs + Reza Zadeh + 2014-01-01 19:53:14 -0800 + Commit: 185c882 + + New documentation + Reza Zadeh + 2014-01-01 19:53:04 -0800 + Commit: dd0d3f0 + + Merge remote-tracking branch 'upstream/master' into sparsesvd + Reza Zadeh + 2014-01-01 18:12:35 -0800 + Commit: 7c04b31 + + large scale considerations + Reza Zadeh + 2013-12-27 04:15:13 -0500 + Commit: ae5102a + + initial large scale testing begin + Reza Zadeh + 2013-12-27 01:51:19 -0500 + Commit: 642ab5c + + cleanup documentation + Reza Zadeh + 2013-12-27 00:41:46 -0500 + Commit: 3369c2d + + add all tests + Reza Zadeh + 2013-12-27 00:36:41 -0500 + Commit: bdb5037 + + test for truncated svd + Reza Zadeh + 2013-12-27 00:34:59 -0500 + Commit: fa1e8d8 + + full rank matrix test added + Reza Zadeh + 2013-12-26 23:21:57 -0500 + Commit: 16de526 + + Main method added for svd + Reza Zadeh + 2013-12-26 18:13:21 -0500 + Commit: fe1a132 + + new main file + Reza Zadeh + 2013-12-26 18:09:33 -0500 + Commit: 1a21ba2 + + Object to hold the svd methods + Reza Zadeh + 2013-12-26 17:39:25 -0500 + Commit: 6c3674c + + Some documentation + Reza Zadeh + 2013-12-26 16:12:40 -0500 + Commit: 6e740cc + + Initial files - no tests + Reza Zadeh + 2013-12-26 15:01:03 -0500 + Commit: 1a173f0 + + +Release 0.9.1 + + Revert "[maven-release-plugin] prepare release v0.9.1-rc2" + Tathagata Das + 2014-03-26 21:53:07 -0700 + Commit: ea5da04 + + Revert "[maven-release-plugin] prepare for next development iteration" + Tathagata Das + 2014-03-26 21:51:40 -0700 + Commit: d16e863 + + [SPARK-1327] GLM needs to check addIntercept for intercept and weights + Xiangrui Meng + 2014-03-26 19:30:57 -0700 + Commit: 4901604, github.com/apache/spark/pull/236 + + SPARK-1322, top in pyspark should sort result in descending order. + Prashant Sharma + 2014-03-26 11:15:02 -0700 + Commit: 2f90dc5, github.com/apache/spark/pull/235 + + [maven-release-plugin] prepare for next development iteration + Ubuntu + 2014-03-26 09:26:45 +0000 + Commit: 1f785d4 + + [maven-release-plugin] prepare release v0.9.1-rc2 + Ubuntu + 2014-03-26 09:26:40 +0000 + Commit: 1197280 + + Updated CHANGES.txt + Tathagata Das + 2014-03-26 02:10:57 -0700 + Commit: 7495dba + + [SPARK-782] Made Spark use existing shaded ASM and removed Spark's ASM dependency + Tathagata Das + 2014-03-25 21:35:36 -0700 + Commit: da87240, github.com/apache/spark/pull/232 + + Revert "[maven-release-plugin] prepare release v0.9.1-rc1" + Tathagata Das + 2014-03-25 15:01:52 -0700 + Commit: 55abe72 + + Revert "[maven-release-plugin] prepare for next development iteration" + Tathagata Das + 2014-03-25 15:01:36 -0700 + Commit: b94f997 + + [maven-release-plugin] prepare for next development iteration + Ubuntu + 2014-03-24 06:56:16 +0000 + Commit: 12e237e + + [maven-release-plugin] prepare release v0.9.1-rc1 + Ubuntu + 2014-03-24 06:56:10 +0000 + Commit: 81c6a06 + + Removed all occurences of incubator from all the pom.xml. + Tathagata Das + 2014-03-23 23:31:59 -0700 + Commit: 60ddb34 + + Updated CHANGES.txt file. + Tathagata Das + 2014-03-23 13:16:50 -0700 + Commit: f176b03 + + Fix to Stage UI to display numbers on progress bar + Emtiaz Ahmed + 2014-03-21 18:07:05 -0700 + Commit: 5e7ac0d, github.com/apache/spark/pull/201 + + SPARK-1284: Fix improper use of SimpleDateFormat + zsxwing + 2014-03-21 16:39:23 -0700 + Commit: 8856076, github.com/apache/spark/pull/179 + + [SPARK-1273] use doi links in mllib-guide + Xiangrui Meng + 2014-03-21 14:35:32 -0700 + Commit: d68549e, github.com/apache/spark/pull/198 + + Removed incubating from Spark version in all the pom.xml. + Tathagata Das + 2014-03-20 18:02:55 -0700 + Commit: 8b1e793 + + Bumped versions to Spark 0.9.1 + Tathagata Das + 2014-03-20 16:55:35 -0700 + Commit: 8a882ef, github.com/apache/spark/pull/190 + + SPARK-1032. If Yarn app fails before registering, app master stays aroun... + Sandy Ryza + 2014-03-20 16:50:44 -0500 + Commit: c6630d3, github.com/apache/spark/pull/28 + + SPARK-1051. On YARN, executors don't doAs submitting user + Sandy Ryza + 2014-03-20 14:48:05 -0500 + Commit: 748f002, github.com/apache/spark/pull/29 + + [SPARK-1285] Backporting updates to streaming docs to branch 0.9 + Aaron Kimball , Tathagata Das , Chen Chao , Andrew Or + 2014-03-20 12:27:47 -0700 + Commit: 1e36690, github.com/apache/spark/pull/183 + + [SPARK-1273] MLlib bug fixes, improvements, and doc updates for v0.9.1 + Xiangrui Meng , Sean Owen , Andrew Tulloch , Chen Chao + 2014-03-19 19:05:26 -0700 + Commit: 1cc979e, github.com/apache/spark/pull/175 + + [SPARK-1275] Made dev/run-tests executable. + Tathagata Das + 2014-03-19 16:10:45 -0700 + Commit: a4eef65, github.com/apache/spark/pull/178 + + Update the yarn alpha version to 0.9.1-incubating-SNAPSHOT + Thomas Graves + 2014-03-19 12:41:11 -0500 + Commit: 72875b2 + + SPARK-1203 fix saving to hdfs from yarn + Thomas Graves + 2014-03-19 08:19:47 -0500 + Commit: 250ec27, github.com/apache/spark/pull/173 + + bugfix: Wrong "Duration" in "Active Stages" in stages page + shiyun.wxm + 2014-03-19 01:42:42 -0700 + Commit: d385b5a, github.com/apache/spark/pull/170 + + [SPARK-1274] Add dev scripts to merge PRs and create releases from master to branch-0.9 + Tathagata Das + 2014-03-18 22:09:16 -0700 + Commit: 7ec78bc, github.com/apache/spark/pull/176 + + Bundle tachyon: SPARK-1269 + Nick Lanham + 2014-03-18 22:05:18 -0700 + Commit: 0183ddd, github.com/apache/spark/pull/137 + + [Spark-1261] add instructions for running python examples to doc overview page + Diana Carroll + 2014-03-17 17:37:03 -0700 + Commit: 20d9458, github.com/apache/spark/pull/162 + + SPARK-1244: Throw exception if map output status exceeds frame size + Patrick Wendell , Andrew Or + 2014-03-17 14:06:28 -0700 + Commit: 4562140, github.com/apache/spark/pull/152 + + SPARK-1240: handle the case of empty RDD when takeSample + CodingCat + 2014-03-16 22:40:22 -0700 + Commit: af7e8b1, github.com/apache/spark/pull/135 + + SPARK-977 Added Python RDD.zip function + Prabin Banka + 2014-03-16 22:16:17 -0700 + Commit: 1dc1e98, github.com/apache/spark/pull/76 + + Spark-1163, Added missing Python RDD functions + prabinb + 2014-03-16 22:14:53 -0700 + Commit: 249930a, github.com/apache/spark/pull/92 + + SPARK-1168, Added foldByKey to pyspark. + Prashant Sharma + 2014-03-16 22:13:33 -0700 + Commit: 4480505, github.com/apache/spark/pull/115 + + Updated link for pyspark examples in docs + Jyotiska NK + 2014-03-16 22:12:51 -0700 + Commit: e74e79a, github.com/apache/spark/pull/22 + + SPARK-1019: pyspark RDD take() throws an NPE + Patrick Wendell + 2014-03-12 23:17:17 -0700 + Commit: ef74e44, github.com/apache/spark/pull/112 + + Fix example bug: compile error + jianghan + 2014-03-12 19:46:48 -0700 + Commit: 87e4dd5, github.com/apache/spark/pull/132 + + SPARK-1162 Added top in python. + Prashant Sharma + 2014-03-12 15:57:54 -0700 + Commit: 51a77e9, github.com/apache/spark/pull/93 + + Version fix in pom file + Patrick Wendell + 2014-03-11 14:48:01 -0700 + Commit: 7049164 + + Log4j build fix on 0.9 branch + Patrick Wendell + 2014-03-11 11:53:29 -0700 + Commit: 6cbd580 + + SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues... + Patrick Wendell + 2014-03-11 11:24:21 -0700 + Commit: 0c91927, github.com/apache/spark/pull/108 + + For outputformats that are Configurable, call setConf before sending data to them. + Bryn Keller + 2014-03-09 17:47:46 -0700 + Commit: 6f0db0a, github.com/apache/spark/pull/638 + + SPARK-1190: Do not initialize log4j if slf4j log4j backend is not being used + Patrick Wendell + 2014-03-08 16:02:56 -0800 + Commit: 0f0d044, github.com/apache/spark/pull/107 + + SPARK-1184: Update the distribution tar.gz to include spark-assembly jar + Mark Grover + 2014-03-05 16:55:36 -0800 + Commit: 0fc0fdb, github.com/apache/spark/pull/78 + + SPARK-1109 wrong API docs for pyspark map function + Prashant Sharma + 2014-03-04 15:33:00 -0800 + Commit: 7ea89ec, github.com/apache/spark/pull/73 + + Add Jekyll tag to isolate "production-only" doc components. (0.9 version) + Patrick Wendell + 2014-03-02 18:18:44 -0800 + Commit: 267d96c, github.com/apache/spark/pull/57 + + Removed reference to incubation in Spark user docs. + Reynold Xin + 2014-02-27 21:14:18 -0800 + Commit: f2bf44a, github.com/apache/spark/pull/2 + + [SPARK-1089] fix the regression problem on ADD_JARS in 0.9 + CodingCat + 2014-02-26 23:42:53 -0800 + Commit: bc5e7d7, github.com/apache/spark/pull/13 + + Removed reference to incubation in README.md. + Reynold Xin + 2014-02-26 16:53:56 -0800 + Commit: 349764d, github.com/apache/spark/pull/1 + + SPARK-1115: Catch depickling errors + Bouke van der Bijl + 2014-02-26 14:53:30 -0800 + Commit: 886a466, github.com/apache/incubator-spark/pull/644 + + SPARK-1135: fix broken anchors in docs + Matei Zaharia + 2014-02-26 11:56:12 -0800 + Commit: 6fe72dd, github.com/apache/spark/pull/3 + + Fix removal from shuffleToMapStage to search for a key-value pair with our stage instead of using our shuffleID. + Matei Zaharia + 2014-02-24 17:01:21 -0800 + Commit: 0661cdc + + SPARK-1124: Fix infinite retries of reduce stage when a map stage failed + Matei Zaharia + 2014-02-24 17:00:47 -0800 + Commit: 5e74b8e + + [SPARK-1055] fix the SCALA_VERSION and SPARK_VERSION in docker file + CodingCat , Nan Zhu + 2014-02-22 15:39:41 -0800 + Commit: 00db30c, github.com/apache/incubator-spark/pull/634 + + SPARK-1117: update accumulator docs + Xiangrui Meng + 2014-02-21 22:44:59 -0800 + Commit: ed58742, github.com/apache/incubator-spark/pull/631 + + [SPARK-1113] External spilling - fix Int.MaxValue hash code collision bug + Andrew Or + 2014-02-21 20:06:09 -0800 + Commit: 84131fe, github.com/apache/incubator-spark/pull/624 + + MLLIB-25: Implicit ALS runs out of memory for moderately large numbers of features + Sean Owen + 2014-02-21 13:39:17 -0800 + Commit: 998abae, github.com/apache/incubator-spark/pull/629 + + SPARK-1111: URL Validation Throws Error for HDFS URL's + Patrick Wendell + 2014-02-21 11:12:38 -0800 + Commit: b3fff96, github.com/apache/incubator-spark/pull/625 + + Super minor: Add require for mergeCombiners in combineByKey + Aaron Davidson + 2014-02-20 16:46:29 -0800 + Commit: 3c44ff4, github.com/apache/incubator-spark/pull/623 + + [SPARK-1105] fix site scala version error in docs + CodingCat + 2014-02-19 15:56:24 -0800 + Commit: 289d761, github.com/apache/incubator-spark/pull/618 + + Revert "[SPARK-1105] fix site scala version error in doc" + Patrick Wendell + 2014-02-18 17:47:34 -0800 + Commit: 7bde72e + + [SPARK-1105] fix site scala version error in doc + CodingCat + 2014-02-18 16:33:41 -0800 + Commit: 0f0395c, github.com/apache/incubator-spark/pull/616 + + Worker registration logging fix + Andrew Ash + 2014-02-17 09:52:16 -0800 + Commit: b0b5288, github.com/apache/incubator-spark/pull/608 + + fix for https://spark-project.atlassian.net/browse/SPARK-1052 + Bijay Bisht + 2014-02-16 16:52:57 -0800 + Commit: e797c1a, github.com/apache/incubator-spark/pull/568 + + Add c3 instance types to Spark EC2 + Christian Lundgren + 2014-02-13 12:44:21 -0800 + Commit: 19b4bb2, github.com/apache/incubator-spark/pull/595 + + SPARK-1088: Create a script for running tests so we can have version specific testing on Jenkins (branch-0.9) + Reynold Xin + 2014-02-12 23:42:58 -0800 + Commit: e5b86b1, github.com/apache/incubator-spark/pull/593 + + Ported hadoopClient jar for < 1.0.1 fix + Bijay Bisht + 2014-02-12 23:42:10 -0800 + Commit: 8093de1, github.com/apache/incubator-spark/pull/584 + + 754bc18 2014-02-12 14:26:39 -0800 + Merge pull request #590 from rxin/scalastyle. + [SPARK-1085: Fix Jenkins pull request builder for branch-0.9 (scalastyle command not found)] + + 28f88c5 2014-02-11 22:43:09 -0800 + Merge pull request #583 from colorant/zookeeper. + [Minor fix for ZooKeeperPersistenceEngine to use configured working dir] + + e70690f 2014-02-09 23:33:35 -0800 + Revert "Merge pull request #560 from pwendell/logging. Closes #560." + [This reverts commit 2e3d1c31db55c7f961e559e47bb497ae15cb74d7.] + + de22abc 2014-02-08 23:37:05 -0800 + Merge pull request #562 from jyotiska/master. Closes #562. + [Added example Python code for sort] + + 2e3d1c3 2014-02-08 23:35:41 -0800 + Merge pull request #560 from pwendell/logging. Closes #560. + [[WIP] SPARK-1067: Default log4j initialization causes errors for those not using log4j] + + 22e0a3b 2014-02-08 13:00:07 -0800 + Merge pull request #561 from Qiuzhuang/master. Closes #561. + [Kill drivers in postStop() for Worker.] + + ce179f6 2014-02-06 22:06:30 -0800 + Merge pull request #533 from andrewor14/master. Closes #533. + [External spilling - generalize batching logic] + + 24e5298 2014-02-06 16:15:36 -0800 + Merge pull request #450 from kayousterhout/fetch_failures. Closes #450. + [Only run ResubmitFailedStages event after a fetch fails] + + 94896bb 2014-02-06 16:13:10 -0800 + Merge pull request #321 from kayousterhout/ui_kill_fix. Closes #321. + [Inform DAG scheduler about all started/finished tasks.] + + 44a2b03 2014-02-05 23:38:25 -0800 + Merge pull request #545 from kayousterhout/fix_progress. Closes #545. + [Fix off-by-one error with task progress info log.] + + b044b0b 2014-02-05 23:37:38 -0800 + Merge pull request #526 from tgravescs/yarn_client_stop_am_fix. Closes #526. + [spark on yarn - yarn-client mode doesn't always exit immediately] + + d815cfa 2014-02-04 09:47:25 -0800 + Merge pull request #534 from sslavic/patch-1. Closes #534. + [Fixed wrong path to compute-classpath.cmd] + + f3cba2d 2014-02-04 09:46:00 -0800 + Merge pull request #535 from sslavic/patch-2. Closes #535. + [Fixed typo in scaladoc] + + 5f63f32 2014-02-03 22:44:30 -0800 + Merge pull request #449 from CrazyJvm/master + [SPARK-1028 : fix "set MASTER automatically fails" bug.] + + 6e4d089 2014-02-03 22:42:43 -0800 + Merge pull request #414 from soulmachine/code-style + [Code clean up for mllib] + + 0021ef9 2014-02-03 22:42:01 -0800 + Merge pull request #445 from kayousterhout/exec_lost + [Fail rather than hanging if a task crashes the JVM.] + + dc8adf1 2014-02-03 22:41:30 -0800 + Merge pull request #489 from ash211/patch-6 + [Clarify spark.default.parallelism] + + 574741f 2014-02-03 22:40:55 -0800 + Merge pull request #493 from kayousterhout/double_add + [Fixed bug where task set managers are added to queue twice] + + 1280e8a 2014-02-03 22:40:29 -0800 + Merge pull request #511 from JoshRosen/SPARK-1040 + [Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)] + + 2c6c9b9 2014-02-03 22:39:59 -0800 + Merge pull request #504 from JoshRosen/SPARK-1025 + [Fix PySpark hang when input files are deleted (SPARK-1025)] + + b10f607 2014-02-03 22:39:10 -0800 + Merge pull request #516 from sarutak/master + [modified SparkPluginBuild.scala to use https protocol for accessing gith...] + + 18520f5 2014-02-03 22:37:38 -0800 + Merge pull request #490 from hsaputra/modify_checkoption_with_isdefined + [Replace the check for None Option with isDefined and isEmpty in Scala code] + + a414071 2014-01-31 16:54:33 -0800 + Merge pull request #524 from rxin/doc + [Added spark.shuffle.file.buffer.kb to configuration doc.] + + a41a83c 2014-01-31 16:53:26 -0800 + Merge pull request #527 from ankurdave/graphx-assembly-pom + [Add GraphX to assembly/pom.xml] + + d18fe1f 2014-01-28 21:55:15 -0800 + Merge pull request #497 from tdas/docs-update + [Updated Spark Streaming Programming Guide] + + 5edbd17 2014-01-28 21:32:58 -0800 + Merge pull request #523 from JoshRosen/SPARK-1043 + [Switch from MUTF8 to UTF8 in PySpark serializers.] + + [maven-release-plugin] prepare for next development iteration + Ubuntu + 2014-01-24 06:15:15 +0000 + Commit: 0f60ef2 + +Release 0.9.0-incubating + + d0a105d Thu Jan 23 20:53:31 2014 -0800 + Merge pull request #505 from JoshRosen/SPARK-1026 + [Deprecate mapPartitionsWithSplit in PySpark (SPARK-1026)] + + e66d4c2 Thu Jan 23 19:47:16 2014 -0800 + Merge pull request #503 from pwendell/master + [Fix bug on read-side of external sort when using Snappy.] + + e8d3f2b Thu Jan 23 19:20:22 2014 -0800 + Merge pull request #502 from pwendell/clone-1 + [Remove Hadoop object cloning and warn users making Hadoop RDD's.] + + 7a62353 Thu Jan 23 19:09:25 2014 -0800 + Merge pull request #501 from JoshRosen/cartesian-rdd-fixes + [Fix two bugs in PySpark cartesian(): SPARK-978 and SPARK-1034] + + 51960b8 Wed Jan 22 19:37:50 2014 -0800 + Merge pull request #496 from pwendell/master + [Fix bug in worker clean-up in UI] + + 828f7b4 Wed Jan 22 15:45:18 2014 -0800 + Merge pull request #495 from srowen/GraphXCommonsMathDependency + [Fix graphx Commons Math dependency] + + dc5857a Wed Jan 22 14:33:25 2014 -0800 + Merge pull request #492 from skicavs/master + [fixed job name and usage information for the JavaSparkPi example] + + dd533c9 Wed Jan 22 14:15:58 2014 -0800 + Merge pull request #478 from sryza/sandy-spark-1033 + [SPARK-1033. Ask for cores in Yarn container requests] + + b6fd3cd Tue Jan 21 00:12:01 2014 -0800 + Merge pull request #480 from pwendell/0.9-fixes + [Handful of 0.9 fixes] + + e5f8917 Mon Jan 20 23:35:07 2014 -0800 + Merge pull request #484 from tdas/run-example-fix + [Made run-example respect SPARK_JAVA_OPTS and SPARK_MEM.] + + 410ba06 Mon Jan 20 22:26:14 2014 -0800 + Merge pull request #482 from tdas/streaming-example-fix + [Added StreamingContext.awaitTermination to streaming examples] + + f137947 Mon Jan 20 22:24:07 2014 -0800 + Merge pull request #483 from pwendell/gitignore + [Restricting /lib to top level directory in .gitignore] + + 94ae25d Sun Jan 19 11:33:51 2014 -0800 + Merge pull request #470 from tgravescs/fix_spark_examples_yarn + [Only log error on missing jar to allow spark examples to jar.] + + 0f077b5 Sun Jan 19 10:30:29 2014 -0800 + Merge pull request #458 from tdas/docs-update + [Updated java API docs for streaming, along with very minor changes in the code examples.] + + 03019d1 Sat Jan 18 16:29:43 2014 -0800 + Merge pull request #459 from srowen/UpdaterL2Regularization + [Correct L2 regularized weight update with canonical form] + + 76147a2 Sat Jan 18 16:24:16 2014 -0800 + Merge pull request #437 from mridulm/master + [Minor api usability changes] + + 4ac8cab Sat Jan 18 16:22:46 2014 -0800 + Merge pull request #426 from mateiz/py-ml-tests + [Re-enable Python MLlib tests (require Python 2.7 and NumPy 1.7+)] + + 34e911c Sat Jan 18 16:17:34 2014 -0800 + Merge pull request #462 from mateiz/conf-file-fix + [Remove Typesafe Config usage and conf files to fix nested property names] + + ff7201c Sat Jan 18 12:50:02 2014 -0800 + Merge pull request #461 from pwendell/master + [Use renamed shuffle spill config in CoGroupedRDD.scala] + + 7b0d5a5 Thu Jan 16 23:18:48 2014 -0800 + Merge pull request #451 from Qiuzhuang/master + [Fixed Window spark shell launch script error.] + + 4ccedb3 Wed Jan 15 14:26:48 2014 -0800 + Merge pull request #444 from mateiz/py-version + [Clarify that Python 2.7 is only needed for MLlib] + + e3fa36f Wed Jan 15 13:56:04 2014 -0800 + Merge pull request #442 from pwendell/standalone + [Workers should use working directory as spark home if it's not specified] + + 29c76d9 Wed Jan 15 13:55:48 2014 -0800 + Merge pull request #443 from tdas/filestream-fix + [Made some classes private[stremaing] and deprecated a method in JavaStreamingContext.] + + aca40aa Wed Jan 15 11:15:47 2014 -0800 + Merge pull request #441 from pwendell/graphx-build + [GraphX shouldn't list Spark as provided.] + + e12c374 Wed Jan 15 10:01:43 2014 -0800 + Merge pull request #433 from markhamstra/debFix + [Updated Debian packaging] + + 2f015c2 Tue Jan 14 23:17:28 2014 -0800 + Merge pull request #436 from ankurdave/VertexId-case + [Rename VertexID -> VertexId in GraphX] + + 2859cab Tue Jan 14 23:08:19 2014 -0800 + Merge pull request #435 from tdas/filestream-fix + [Fixed the flaky tests by making SparkConf not serializable] + + fbfbb33 Tue Jan 14 23:06:29 2014 -0800 + Merge pull request #434 from rxin/graphxmaven + [Fixed SVDPlusPlusSuite in Maven build.] + + 2c6c07f Tue Jan 14 21:53:05 2014 -0800 + Merge pull request #424 from jegonzal/GraphXProgrammingGuide + [Additional edits for clarity in the graphx programming guide.] + + 6fa4e02 Tue Jan 14 21:51:25 2014 -0800 + Merge pull request #431 from ankurdave/graphx-caching-doc + [Describe caching and uncaching in GraphX programming guide] + + 2f930d5 Tue Jan 14 15:00:11 2014 -0800 + Merge pull request #428 from pwendell/writeable-objects + [Don't clone records for text files] + + 329c9df Tue Jan 14 14:53:36 2014 -0800 + Merge pull request #429 from ankurdave/graphx-examples-pom.xml + [Add GraphX dependency to examples/pom.xml] + + a14933d Tue Jan 14 14:52:42 2014 -0800 + Merge pull request #427 from pwendell/deprecate-aggregator + [Deprecate rather than remove old combineValuesByKey function] + + 119b6c5 Tue Jan 14 13:29:08 2014 -0800 + Merge pull request #425 from rxin/scaladoc + [API doc update & make Broadcast public] + + bf3b150 Tue Jan 14 09:45:22 2014 -0800 + Merge pull request #423 from jegonzal/GraphXProgrammingGuide + [Improving the graphx-programming-guide] + + 1b4adc2 Tue Jan 14 01:19:24 2014 -0800 + Merge pull request #420 from pwendell/header-files + [Add missing header files] + + b60840e Tue Jan 14 00:48:34 2014 -0800 + Merge pull request #418 from pwendell/0.9-versions + [Version changes for release 0.9.0.] + + 980250b Tue Jan 14 00:05:37 2014 -0800 + Merge pull request #416 from tdas/filestream-fix + [Removed unnecessary DStream operations and updated docs] + + 055be5c Mon Jan 13 23:26:44 2014 -0800 + Merge pull request #415 from pwendell/shuffle-compress + [Enable compression by default for spills] + + fdaabdc Mon Jan 13 23:08:26 2014 -0800 + Merge pull request #380 from mateiz/py-bayes + [Add Naive Bayes to Python MLlib, and some API fixes] + + 4a805af Mon Jan 13 22:58:38 2014 -0800 + Merge pull request #367 from ankurdave/graphx + [GraphX: Unifying Graphs and Tables] + + 945fe7a Mon Jan 13 22:56:12 2014 -0800 + Merge pull request #408 from pwendell/external-serializers + [Improvements to external sorting] + + 68641bc Mon Jan 13 22:54:13 2014 -0800 + Merge pull request #413 from rxin/scaladoc + [Adjusted visibility of various components and documentation for 0.9.0 release.] + + 0ca0d4d Mon Jan 13 22:32:21 2014 -0800 + Merge pull request #401 from andrewor14/master + [External sorting - Add number of bytes spilled to Web UI] + + 08b9fec Mon Jan 13 22:29:03 2014 -0800 + Merge pull request #409 from tdas/unpersist + [Automatically unpersisting RDDs that have been cleaned up from DStreams] + + b07bc02 Mon Jan 13 20:45:22 2014 -0800 + Merge pull request #412 from harveyfeng/master + [Add default value for HadoopRDD's `cloneRecords` constructor arg] + + a2fee38 Mon Jan 13 19:45:26 2014 -0800 + Merge pull request #411 from tdas/filestream-fix + [Improved logic of finding new files in FileInputDStream] + + 01c0d72 Mon Jan 13 16:24:30 2014 -0800 + Merge pull request #410 from rxin/scaladoc1 + [Updated JavaStreamingContext to make scaladoc compile.] + + 8038da2 Mon Jan 13 14:59:30 2014 -0800 + Merge pull request #2 from jegonzal/GraphXCCIssue + [Improving documentation and identifying potential bug in CC calculation.] + + b93f9d4 Mon Jan 13 12:18:05 2014 -0800 + Merge pull request #400 from tdas/dstream-move + [Moved DStream and PairDSream to org.apache.spark.streaming.dstream] + + e6ed13f Sun Jan 12 22:35:14 2014 -0800 + Merge pull request #397 from pwendell/host-port + [Remove now un-needed hostPort option] + + 0b96d85 Sun Jan 12 21:31:43 2014 -0800 + Merge pull request #399 from pwendell/consolidate-off + [Disable shuffle file consolidation by default] + + 0ab505a Sun Jan 12 21:31:04 2014 -0800 + Merge pull request #395 from hsaputra/remove_simpleredundantreturn_scala + [Remove simple redundant return statements for Scala methods/functions] + + 405bfe8 Sun Jan 12 20:04:21 2014 -0800 + Merge pull request #394 from tdas/error-handling + [Better error handling in Spark Streaming and more API cleanup] + + 28a6b0c Sun Jan 12 19:49:36 2014 -0800 + Merge pull request #398 from pwendell/streaming-api + [Rename DStream.foreach to DStream.foreachRDD] + + 074f502 Sun Jan 12 17:01:13 2014 -0800 + Merge pull request #396 from pwendell/executor-env + [Setting load defaults to true in executor] + + 82e2b92 Sun Jan 12 16:55:11 2014 -0800 + Merge pull request #392 from rxin/listenerbus + [Stop SparkListenerBus daemon thread when DAGScheduler is stopped.] + + 288a878 Sat Jan 11 21:53:19 2014 -0800 + Merge pull request #389 from rxin/clone-writables + [Minor update for clone writables and more documentation.] + + dbc11df Sat Jan 11 18:07:13 2014 -0800 + Merge pull request #388 from pwendell/master + [Fix UI bug introduced in #244.] + + 409866b Sat Jan 11 17:12:06 2014 -0800 + Merge pull request #393 from pwendell/revert-381 + [Revert PR 381] + + 6510f04 Sat Jan 11 12:48:26 2014 -0800 + Merge pull request #387 from jerryshao/conf-fix + [Fix configure didn't work small problem in ALS] + + ee6e7f9 Sat Jan 11 12:07:55 2014 -0800 + Merge pull request #359 from ScrapCodes/clone-writables + [We clone hadoop key and values by default and reuse objects if asked to.] + + 4216178 Sat Jan 11 09:46:48 2014 -0800 + Merge pull request #373 from jerryshao/kafka-upgrade + [Upgrade Kafka dependecy to 0.8.0 release version] + + 92ad18b Fri Jan 10 23:25:15 2014 -0800 + Merge pull request #376 from prabeesh/master + [Change clientId to random clientId] + + 0b5ce7a Fri Jan 10 23:23:21 2014 -0800 + Merge pull request #386 from pwendell/typo-fix + [Small typo fix] + + 1d7bef0 Fri Jan 10 18:53:03 2014 -0800 + Merge pull request #381 from mateiz/default-ttl + [Fix default TTL for metadata cleaner] + + 44d6a8e Fri Jan 10 17:51:50 2014 -0800 + Merge pull request #382 from RongGu/master + [Fix a type error in comment lines] + + 88faa30 Fri Jan 10 17:14:22 2014 -0800 + Merge pull request #385 from shivaram/add-i2-instances + [Add i2 instance types to Spark EC2.] + + f265531 Fri Jan 10 16:25:44 2014 -0800 + Merge pull request #383 from tdas/driver-test + [API for automatic driver recovery for streaming programs and other bug fixes] + + d37408f Fri Jan 10 16:25:01 2014 -0800 + Merge pull request #377 from andrewor14/master + [External Sorting for Aggregator and CoGroupedRDDs (Revisited)] + + 0eaf01c Fri Jan 10 15:32:19 2014 -0800 + Merge pull request #369 from pillis/master + [SPARK-961 Add a Vector.random() method] + + 7cef843 Fri Jan 10 15:34:15 2014 -0600 + Merge pull request #371 from tgravescs/yarn_client_addjar_misc_fixes + [Yarn client addjar and misc fixes] + + 7b58f11 Fri Jan 10 12:47:46 2014 -0800 + Merge pull request #384 from pwendell/debug-logs + [Make DEBUG-level logs consummable.] + + 23d2995 Fri Jan 10 10:20:02 2014 -0800 + Merge pull request #1 from jegonzal/graphx + [ProgrammingGuide] + + 0ebc973 Thu Jan 9 23:58:49 2014 -0800 + Merge pull request #375 from mateiz/option-fix + [Fix bug added when we changed AppDescription.maxCores to an Option] + + dd03cea Thu Jan 9 23:38:03 2014 -0800 + Merge pull request #378 from pwendell/consolidate_on + [Enable shuffle consolidation by default.] + + 997c830 Thu Jan 9 22:22:20 2014 -0800 + Merge pull request #363 from pwendell/streaming-logs + [Set default logging to WARN for Spark streaming examples.] + + 300eaa9 Thu Jan 9 20:29:51 2014 -0800 + Merge pull request #353 from pwendell/ipython-simplify + [Simplify and fix pyspark script.] + + 4b074fa Thu Jan 9 19:03:55 2014 -0800 + Merge pull request #374 from mateiz/completeness + [Add some missing Java API methods] + + a9d5333 Thu Jan 9 18:46:46 2014 -0800 + Merge pull request #294 from RongGu/master + [Bug fixes for updating the RDD block's memory and disk usage information] + + d86a85e Thu Jan 9 18:37:52 2014 -0800 + Merge pull request #293 from pwendell/standalone-driver + [SPARK-998: Support Launching Driver Inside of Standalone Mode] + + 26cdb5f Thu Jan 9 17:16:34 2014 -0800 + Merge pull request #372 from pwendell/log4j-fix-1 + [Send logs to stderr by default (instead of stdout).] + + 12f414e Thu Jan 9 15:31:30 2014 -0800 + Merge pull request #362 from mateiz/conf-getters + [Use typed getters for configuration settings] + + 365cac9 Thu Jan 9 00:56:16 2014 -0800 + Merge pull request #361 from rxin/clean + [Minor style cleanup. Mostly on indenting & line width changes.] + + 73c724e Thu Jan 9 00:32:19 2014 -0800 + Merge pull request #368 from pwendell/sbt-fix + [Don't delegate to users `sbt`.] + + dceedb4 Wed Jan 8 23:19:28 2014 -0800 + Merge pull request #364 from pwendell/fix + [Fixing config option "retained_stages" => "retainedStages".] + + 04d83fc Wed Jan 8 11:55:37 2014 -0800 + Merge pull request #360 from witgo/master + [fix make-distribution.sh show version: command not found] + + 56ebfea Wed Jan 8 11:50:06 2014 -0800 + Merge pull request #357 from hsaputra/set_boolean_paramname + [Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID] + + bdeaeaf Wed Jan 8 11:48:39 2014 -0800 + Merge pull request #358 from pwendell/add-cdh + [Add CDH Repository to Maven Build] + + 5cae05f Wed Jan 8 11:47:28 2014 -0800 + Merge pull request #356 from hsaputra/remove_deprecated_cleanup_method + [Remove calls to deprecated mapred's OutputCommitter.cleanupJob] + + 6eef78d Wed Jan 8 08:49:20 2014 -0600 + Merge pull request #345 from colorant/yarn + [support distributing extra files to worker for yarn client mode] + + bb6a39a Tue Jan 7 22:32:18 2014 -0800 + Merge pull request #322 from falaki/MLLibDocumentationImprovement + [SPARK-1009 Updated MLlib docs to show how to use it in Python] + + cb1b927 Tue Jan 7 22:26:28 2014 -0800 + Merge pull request #355 from ScrapCodes/patch-1 + [Update README.md] + + c0f0155 Tue Jan 7 22:21:52 2014 -0800 + Merge pull request #313 from tdas/project-refactor + [Refactored the streaming project to separate external libraries like Twitter, Kafka, Flume, etc.] + + f5f12dc Tue Jan 7 21:56:35 2014 -0800 + Merge pull request #336 from liancheng/akka-remote-lookup + [Get rid of `Either[ActorRef, ActorSelection]'] + + 11891e6 Wed Jan 8 00:32:18 2014 -0500 + Merge pull request #327 from lucarosellini/master + [Added ‘-i’ command line option to Spark REPL] + + 7d0aac9 Wed Jan 8 00:30:45 2014 -0500 + Merge pull request #354 from hsaputra/addasfheadertosbt + [Add ASF header to the new sbt script.] + + d75dc42 Wed Jan 8 00:30:03 2014 -0500 + Merge pull request #350 from mateiz/standalone-limit + [Add way to limit default # of cores used by apps in standalone mode] + + 61674bc Tue Jan 7 18:32:13 2014 -0800 + Merge pull request #352 from markhamstra/oldArch + [Don't leave os.arch unset after BlockManagerSuite] + + b2e690f Tue Jan 7 16:57:08 2014 -0800 + Merge pull request #328 from falaki/MatrixFactorizationModel-fix + [SPARK-1012: DAGScheduler Exception Fix] + + 6ccf8ce Tue Jan 7 15:49:14 2014 -0800 + Merge pull request #351 from pwendell/maven-fix + [Add log4j exclusion rule to maven.] + + 7d5fa17 Tue Jan 7 11:31:34 2014 -0800 + Merge pull request #337 from yinxusen/mllib-16-bugfix + [Mllib 16 bugfix] + + 71fc113 Tue Jan 7 11:30:35 2014 -0800 + Merge pull request #349 from CodingCat/support-worker_dir + [add the comments about SPARK_WORKER_DIR] + + 15d9534 Tue Jan 7 08:10:02 2014 -0800 + Merge pull request #318 from srowen/master + [Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance] + + 468af0f Tue Jan 7 08:09:01 2014 -0800 + Merge pull request #348 from prabeesh/master + [spark -> org.apache.spark] + + c3cf047 Tue Jan 7 00:54:25 2014 -0800 + Merge pull request #339 from ScrapCodes/conf-improvements + [Conf improvements] + + a862caf Tue Jan 7 00:18:20 2014 -0800 + Merge pull request #331 from holdenk/master + [Add a script to download sbt if not present on the system] + + b97ef21 Mon Jan 6 20:12:57 2014 -0800 + Merge pull request #346 from sproblvem/patch-1 + [Update stop-slaves.sh] + + 7210257 Mon Jan 6 18:25:44 2014 -0800 + Merge pull request #128 from adamnovak/master + [Fix failing "sbt/sbt publish-local" by adding a no-argument PrimitiveKeyOpenHashMap constructor ] + + e4d6057 Mon Jan 6 14:56:54 2014 -0800 + Merge pull request #343 from pwendell/build-fix + [Fix test breaking downstream builds] + + 93bf962 Mon Jan 6 11:42:41 2014 -0800 + Merge pull request #340 from ScrapCodes/sbt-fixes + [Made java options to be applied during tests so that they become self explanatory.] + + 60edeb3 Mon Jan 6 11:40:32 2014 -0800 + Merge pull request #338 from ScrapCodes/ning-upgrade + [SPARK-1005 Ning upgrade] + + c708e81 Mon Jan 6 11:35:48 2014 -0800 + Merge pull request #341 from ash211/patch-5 + [Clarify spark.cores.max in docs] + + 33fcb91 Mon Jan 6 11:19:23 2014 -0800 + Merge pull request #342 from tgravescs/fix_maven_protobuf + [Change protobuf version for yarn alpha back to 2.4.1] + + 357083c Mon Jan 6 10:29:04 2014 -0800 + Merge pull request #330 from tgravescs/fix_addjars_null_handling + [Fix handling of empty SPARK_EXAMPLES_JAR] + + a2e7e04 Sun Jan 5 22:37:36 2014 -0800 + Merge pull request #333 from pwendell/logging-silence + [Quiet ERROR-level Akka Logs] + + 5b0986a Sun Jan 5 19:25:09 2014 -0800 + Merge pull request #334 from pwendell/examples-fix + [Removing SPARK_EXAMPLES_JAR in the code] + + f4b924f Sun Jan 5 17:11:47 2014 -0800 + Merge pull request #335 from rxin/ser + [Fall back to zero-arg constructor for Serializer initialization if there is no constructor that accepts SparkConf.] + + d43ad3e Sat Jan 4 16:29:30 2014 -0800 + Merge pull request #292 from soulmachine/naive-bayes + [standard Naive Bayes classifier] + + 86404da Sat Jan 4 14:55:54 2014 -0800 + Merge pull request #127 from jegonzal/MapByPartition + [Adding mapEdges and mapTriplets by Partition] + + e68cdb1 Sat Jan 4 13:46:02 2014 -0800 + Merge pull request #124 from jianpingjwang/master + [refactor and bug fix] + + 280ddf6 Sat Jan 4 12:54:41 2014 -0800 + Merge pull request #121 from ankurdave/more-simplify + [Simplify GraphImpl internals further] + + 10fe23b Fri Jan 3 23:50:14 2014 -0800 + Merge pull request #329 from pwendell/remove-binaries + [SPARK-1002: Remove Binaries from Spark Source] + + c4d6145 Fri Jan 3 16:30:53 2014 -0800 + Merge pull request #325 from witgo/master + [Modify spark on yarn to create SparkConf process] + + 4ae101f Fri Jan 3 11:24:35 2014 -0800 + Merge pull request #317 from ScrapCodes/spark-915-segregate-scripts + [Spark-915 segregate scripts] + + 87248bd Fri Jan 3 00:45:31 2014 -0800 + Merge pull request #1 from apache/master + [Merge latest Spark changes] + + 30b9db0 Thu Jan 2 23:15:55 2014 -0800 + Merge pull request #285 from colorant/yarn-refactor + [Yarn refactor] + + 498a5f0 Thu Jan 2 19:06:40 2014 -0800 + Merge pull request #323 from tgravescs/sparkconf_yarn_fix + [fix spark on yarn after the sparkConf changes] + + 0475ca8 Thu Jan 2 15:17:08 2014 -0800 + Merge pull request #320 from kayousterhout/erroneous_failed_msg + [Remove erroneous FAILED state for killed tasks.] + + 588a169 Thu Jan 2 13:20:54 2014 -0800 + Merge pull request #297 from tdas/window-improvement + [Improvements to DStream window ops and refactoring of Spark's CheckpointSuite] + + 5e67cdc Thu Jan 2 12:56:28 2014 -0800 + Merge pull request #319 from kayousterhout/remove_error_method + [Removed redundant TaskSetManager.error() function.] + + ca67909 Thu Jan 2 15:54:54 2014 -0500 + Merge pull request #311 from tmyklebu/master + [SPARK-991: Report information gleaned from a Python stacktrace in the UI] + + 3713f81 Wed Jan 1 21:29:12 2014 -0800 + Merge pull request #309 from mateiz/conf2 + [SPARK-544. Migrate configuration to a SparkConf class] + + c1d928a Wed Jan 1 17:03:48 2014 -0800 + Merge pull request #312 from pwendell/log4j-fix-2 + [SPARK-1008: Logging improvments] + + dc9cb83 Wed Jan 1 13:28:34 2014 -0800 + Merge pull request #126 from jegonzal/FixingPersist + [Fixing Persist Behavior] + + 9a0ff72 Tue Dec 31 21:50:24 2013 -0800 + Merge pull request #314 from witgo/master + [restore core/pom.xml file modification] + + 8b8e70e Tue Dec 31 17:48:24 2013 -0800 + Merge pull request #73 from falaki/ApproximateDistinctCount + [Approximate distinct count] + + 63b411d Tue Dec 31 14:31:28 2013 -0800 + Merge pull request #238 from ngbinh/upgradeNetty + [upgrade Netty from 4.0.0.Beta2 to 4.0.13.Final] + + 32d6ae9 Tue Dec 31 13:51:07 2013 -0800 + Merge pull request #120 from ankurdave/subgraph-reuses-view + [Reuse VTableReplicated in GraphImpl.subgraph] + + 55b7e2f Tue Dec 31 10:12:51 2013 -0800 + Merge pull request #289 from tdas/filestream-fix + [Bug fixes for file input stream and checkpointing] + + 2b71ab9 Mon Dec 30 11:01:30 2013 -0800 + Merge pull request from aarondav: Utilize DiskBlockManager pathway for temp file writing + [This gives us a couple advantages:] + + 50e3b8e Mon Dec 30 07:44:26 2013 -0800 + Merge pull request #308 from kayousterhout/stage_naming + [Changed naming of StageCompleted event to be consistent] + + 72a17b6 Sat Dec 28 21:25:40 2013 -1000 + Revert "Merge pull request #310 from jyunfan/master" + [This reverts commit 79b20e4dbe3dcd8559ec8316784d3334bb55868b, reversing] + + 79b20e4 Sat Dec 28 21:13:36 2013 -1000 + Merge pull request #310 from jyunfan/master + [Fix typo in the Accumulators section] + + 7375047 Sat Dec 28 13:25:06 2013 -0800 + Merge pull request #304 from kayousterhout/remove_unused + [Removed unused failed and causeOfFailure variables (in TaskSetManager)] + + ad3dfd1 Fri Dec 27 22:10:14 2013 -0500 + Merge pull request #307 from kayousterhout/other_failure + [Removed unused OtherFailure TaskEndReason.] + + b579b83 Fri Dec 27 22:09:04 2013 -0500 + Merge pull request #306 from kayousterhout/remove_pending + [Remove unused hasPendingTasks methods] + + 19672dc Fri Dec 27 13:37:10 2013 -0800 + Merge pull request #305 from kayousterhout/line_spacing + [Fixed >100char lines in DAGScheduler.scala] + + 7be1e57 Thu Dec 26 23:41:40 2013 -1000 + Merge pull request #298 from aarondav/minor + [Minor: Decrease margin of left side of Log page] + + 7d811ba Thu Dec 26 23:39:58 2013 -1000 + Merge pull request #302 from pwendell/SPARK-1007 + [SPARK-1007: spark-class2.cmd should change SCALA_VERSION to be 2.10] + + 5e69fc5 Thu Dec 26 19:10:39 2013 -0500 + Merge pull request #295 from markhamstra/JobProgressListenerNPE + [Avoid a lump of coal (NPE) in JobProgressListener's stocking.] + + da20270 Thu Dec 26 12:11:52 2013 -0800 + Merge pull request #1 from aarondav/driver + [Refactor DriverClient to be more Actor-based] + + e240bad Thu Dec 26 12:30:48 2013 -0500 + Merge pull request #296 from witgo/master + [Renamed ClusterScheduler to TaskSchedulerImpl for yarn and new-yarn package] + + c344ed0 Thu Dec 26 01:31:06 2013 -0500 + Merge pull request #283 from tmyklebu/master + [Python bindings for mllib] + + 56094bc Wed Dec 25 13:14:33 2013 -0500 + Merge pull request #290 from ash211/patch-3 + [Typo: avaiable -> available] + + 4842a07 Wed Dec 25 01:52:15 2013 -0800 + Merge pull request #287 from azuryyu/master + [Fixed job name in the java streaming example.] + + 85a344b Tue Dec 24 16:35:06 2013 -0800 + Merge pull request #127 from kayousterhout/consolidate_schedulers + [Deduplicate Local and Cluster schedulers.] + + c2dd6bc Tue Dec 24 14:36:47 2013 -0800 + Merge pull request #279 from aarondav/shuffle-cleanup0 + [Clean up shuffle files once their metadata is gone] + + 3bf7c70 Tue Dec 24 16:37:13 2013 -0500 + Merge pull request #275 from ueshin/wip/changeclasspathorder + [Change the order of CLASSPATH.] + + d63856c Mon Dec 23 22:07:26 2013 -0800 + Merge pull request #286 from rxin/build + [Show full stack trace and time taken in unit tests.] + + 23a9ae6 Tue Dec 24 00:08:48 2013 -0500 + Merge pull request #277 from tdas/scheduler-update + [Refactored the streaming scheduler and added StreamingListener interface] + + 11107c9 Mon Dec 23 10:38:20 2013 -0800 + Merge pull request #244 from leftnoteasy/master + [Added SPARK-968 implementation for review] + + 44e4205 Sun Dec 22 11:44:18 2013 -0800 + Merge pull request #116 from jianpingjwang/master + [remove unused variables and fix a bug] + + 4797c22 Fri Dec 20 13:30:39 2013 -0800 + Merge pull request #118 from ankurdave/VertexPartitionSuite + [Test VertexPartition and fix bugs] + + 0bc57c5 Fri Dec 20 11:56:54 2013 -0800 + Merge pull request #280 from aarondav/minor + [Minor cleanup for standalone scheduler] + + ac70b8f Fri Dec 20 10:56:10 2013 -0800 + Merge pull request #117 from ankurdave/more-tests + [More tests] + + 45310d4 Thu Dec 19 22:08:20 2013 -0800 + Merge pull request #115 from ankurdave/test-reorg + [Reorganize unit tests; improve GraphSuite test coverage] + + 9228ec8 Thu Dec 19 21:37:15 2013 -0800 + Merge pull request #1 from aarondav/127 + [Merge master into 127] + + eca68d4 Thu Dec 19 18:12:22 2013 -0800 + Merge pull request #272 from tmyklebu/master + [Track and report task result serialisation time.] + + 7990c56 Thu Dec 19 13:35:09 2013 -0800 + Merge pull request #276 from shivaram/collectPartition + [Add collectPartition to JavaRDD interface.] + + 440e531 Thu Dec 19 10:38:56 2013 -0800 + Merge pull request #278 from MLnick/java-python-tostring + [Add toString to Java RDD, and __repr__ to Python RDD] + + d8d3f3e Thu Dec 19 00:06:43 2013 -0800 + Merge pull request #183 from aarondav/spark-959 + [[SPARK-959] Explicitly depend on org.eclipse.jetty.orbit jar] + + bfba532 Wed Dec 18 22:22:21 2013 -0800 + Merge pull request #247 from aarondav/minor + [Increase spark.akka.askTimeout default to 30 seconds] + + da301b5 Wed Dec 18 20:03:29 2013 -0800 + Merge pull request #112 from amatsukawa/scc + [Strongly connected component algorithm] + + c64a53a Wed Dec 18 16:56:26 2013 -0800 + Merge pull request #267 from JoshRosen/cygwin + [Fix Cygwin support in several scripts.] + + a645ef6 Wed Dec 18 16:07:52 2013 -0800 + Merge pull request #48 from amatsukawa/add_project_to_graph + [Add mask operation on graph and filter graph primitive] + + d7ebff0 Wed Dec 18 15:38:48 2013 -0800 + Merge pull request #1 from ankurdave/add_project_to_graph + [Merge current master and reimplement Graph.mask using innerJoin] + + 5ea1872 Wed Dec 18 15:27:24 2013 -0800 + Merge pull request #274 from azuryy/master + [Fixed the example link in the Scala programing guid.] + + 3fd2e09 Wed Dec 18 12:52:36 2013 -0800 + Merge pull request #104 from jianpingjwang/master + [SVD++ demo] + + f4effb3 Tue Dec 17 22:26:21 2013 -0800 + Merge pull request #273 from rxin/top + [Fixed a performance problem in RDD.top and BoundedPriorityQueue] + + 1b5eacb Tue Dec 17 13:49:17 2013 -0800 + Merge pull request #102 from ankurdave/clustered-edge-index + [Add clustered index on edges by source vertex] + + 7a8169b Mon Dec 16 22:42:21 2013 -0800 + Merge pull request #268 from pwendell/shaded-protobuf + [Add support for 2.2. to master (via shaded jars)] + + 0476c84 Mon Dec 16 17:19:25 2013 -0800 + Merge pull request #100 from ankurdave/mrTriplets-active-set + [Support activeSet option in mapReduceTriplets] + + 964a3b6 Mon Dec 16 15:23:51 2013 -0800 + Merge pull request #270 from ewencp/really-force-ssh-pseudo-tty-master + [Force pseudo-tty allocation in spark-ec2 script.] + + 5192ef3 Mon Dec 16 15:08:08 2013 -0800 + Merge pull request #94 from ankurdave/load-edges-columnar + [Load edges in columnar format] + + 883e034 Mon Dec 16 14:16:02 2013 -0800 + Merge pull request #245 from gregakespret/task-maxfailures-fix + [Fix for spark.task.maxFailures not enforced correctly.] + + a51f340 Sun Dec 15 22:02:30 2013 -0800 + Merge pull request #265 from markhamstra/scala.binary.version + [DRY out the POMs with scala.binary.version] + + ded10ce Sun Dec 15 17:25:33 2013 -0800 + Merge pull request #103 from amplab/optimizations + [Optimizations cherry-picked from SIGMOD branches] + + d2ced6d Sun Dec 15 14:11:34 2013 -0800 + Merge pull request #256 from MLnick/master + [Fix 'IPYTHON=1 ./pyspark' throwing ValueError] + + c55e698 Sun Dec 15 12:49:02 2013 -0800 + Merge pull request #257 from tgravescs/sparkYarnFixName + [Fix the --name option for Spark on Yarn] + + ab85f88 Sun Dec 15 12:48:32 2013 -0800 + Merge pull request #264 from shivaram/spark-class-fix + [Use CoarseGrainedExecutorBackend in spark-class] + + 8a56c1f Sat Dec 14 16:29:24 2013 -0800 + Merge pull request #84 from amatsukawa/graphlab_enhancements + [GraphLab bug fix & set start vertex] + + 7db9165 Sat Dec 14 14:16:34 2013 -0800 + Merge pull request #251 from pwendell/master + [Fix list rendering in YARN markdown docs.] + + 2fd781d Sat Dec 14 12:59:37 2013 -0800 + Merge pull request #249 from ngbinh/partitionInJavaSortByKey + [Expose numPartitions parameter in JavaPairRDD.sortByKey()] + + 9bf192b Sat Dec 14 12:52:18 2013 -0800 + Merge pull request #91 from amplab/standalone-pagerank + [Standalone PageRank] + + 840af5e Sat Dec 14 12:51:51 2013 -0800 + Merge pull request #99 from ankurdave/only-dynamic-pregel + [Remove static Pregel; take maxIterations in dynamic Pregel] + + 97ac060 Sat Dec 14 00:22:45 2013 -0800 + Merge pull request #259 from pwendell/scala-2.10 + [Migration to Scala 2.10] + + 7ac944f Fri Dec 13 23:22:08 2013 -0800 + Merge pull request #262 from pwendell/mvn-fix + [Fix maven build issues in 2.10 branch] + + 6defb06 Fri Dec 13 21:18:57 2013 -0800 + Merge pull request #261 from ScrapCodes/scala-2.10 + [Added a comment about ActorRef and ActorSelection difference.] + + 76566b1 Fri Dec 13 10:11:02 2013 -0800 + Merge pull request #260 from ScrapCodes/scala-2.10 + [Review comments on the PR for scala 2.10 migration.] + + 0aeb182 Thu Dec 12 21:14:42 2013 -0800 + Merge pull request #255 from ScrapCodes/scala-2.10 + [Disabled yarn 2.2 in sbt and mvn build and added a message in the sbt build.] + + 2e89398 Wed Dec 11 23:10:53 2013 -0800 + Merge pull request #254 from ScrapCodes/scala-2.10 + [Scala 2.10 migration] + + ce6ca4e Wed Dec 11 22:30:54 2013 -0800 + Merge pull request #97 from dcrankshaw/fix-rddtop + [Added BoundedPriorityQueue kryo registrator. Fixes top issue.] + + d2efe13 Tue Dec 10 13:01:26 2013 -0800 + Merge pull request #250 from pwendell/master + [README incorrectly suggests build sources spark-env.sh] + + 6169fe1 Mon Dec 9 16:51:36 2013 -0800 + Merge pull request #246 from pwendell/master + [Add missing license headers] + + d992ec6 Sun Dec 8 20:49:20 2013 -0800 + Merge pull request #195 from dhardy92/fix_DebScriptPackage + [[Deb] fix package of Spark classes adding org.apache prefix in scripts embeded in .deb] + + 1f4a4bc Sat Dec 7 22:34:34 2013 -0800 + Merge pull request #242 from pwendell/master + [Update broken links and add HDP 2.0 version string] + + 6494d62 Sat Dec 7 11:56:16 2013 -0800 + Merge pull request #240 from pwendell/master + [SPARK-917 Improve API links in nav bar] + + f466f79 Sat Dec 7 11:51:52 2013 -0800 + Merge pull request #239 from aarondav/nit + [Correct spellling error in configuration.md] + + 3abfbfb Sat Dec 7 11:24:19 2013 -0800 + Merge pull request #92 from ankurdave/rdd-names + [Set RDD names for easy debugging] + + 31e8a14 Fri Dec 6 21:49:55 2013 -0800 + Merge pull request #90 from amplab/pregel-replicate-changed + [Replicate only changed vertices] + + 10c3c0c Fri Dec 6 20:29:45 2013 -0800 + Merge pull request #237 from pwendell/formatting-fix + [Formatting fix] + + 1b38f5f Fri Dec 6 20:16:15 2013 -0800 + Merge pull request #236 from pwendell/shuffle-docs + [Adding disclaimer for shuffle file consolidation] + + e5d5728 Fri Dec 6 20:14:56 2013 -0800 + Merge pull request #235 from pwendell/master + [Minor doc fixes and updating README] + + 241336a Fri Dec 6 17:29:03 2013 -0800 + Merge pull request #234 from alig/master + [Updated documentation about the YARN v2.2 build process] + + e039234 Fri Dec 6 11:49:59 2013 -0800 + Merge pull request #190 from markhamstra/Stages4Jobs + [stageId <--> jobId mapping in DAGScheduler] + + bfa6860 Fri Dec 6 11:04:03 2013 -0800 + Merge pull request #233 from hsaputra/changecontexttobackend + [Change the name of input argument in ClusterScheduler#initialize from context to backend.] + + 3fb302c Fri Dec 6 11:03:32 2013 -0800 + Merge pull request #205 from kayousterhout/logging + [Added logging of scheduler delays to UI] + + 87676a6 Fri Dec 6 11:01:42 2013 -0800 + Merge pull request #220 from rxin/zippart + [Memoize preferred locations in ZippedPartitionsBaseRDD] + + 0780498 Thu Dec 5 23:29:42 2013 -0800 + Merge pull request #232 from markhamstra/FiniteWait + [jobWaiter.synchronized before jobWaiter.wait] + + 1c8500e Thu Dec 5 16:25:44 2013 -0800 + Merge pull request #88 from amplab/varenc + [Fixed a bug that variable encoding doesn't work for ints that use all 64 bits.] + + e0bcaa0 Thu Dec 5 12:37:02 2013 -0800 + Merge pull request #86 from ankurdave/vid-varenc + [Finish work on #85] + + 5d46025 Thu Dec 5 12:31:24 2013 -0800 + Merge pull request #228 from pwendell/master + [Document missing configs and set shuffle consolidation to false.] + + 3e96b9a Thu Dec 5 12:07:36 2013 -0800 + Merge pull request #85 from ankurdave/vid-varenc + [Always write Vids using variable encoding] + + 72b6961 Wed Dec 4 23:33:04 2013 -0800 + Merge pull request #199 from harveyfeng/yarn-2.2 + [Hadoop 2.2 migration] + + e0347ba Wed Dec 4 17:38:06 2013 -0800 + Merge pull request #83 from ankurdave/fix-tests + [Fix compile errors in GraphSuite and SerializerSuite] + + 182f9ba Wed Dec 4 15:52:07 2013 -0800 + Merge pull request #227 from pwendell/master + [Fix small bug in web UI and minor clean-up.] + + cbd3b75 Wed Dec 4 15:35:26 2013 -0800 + Merge pull request #81 from amplab/clean1 + [Codebase refactoring] + + b9e7609 Wed Dec 4 14:42:09 2013 -0800 + Merge pull request #225 from ash211/patch-3 + [Add missing space after "Serialized" in StorageLevel] + + 055462c Wed Dec 4 14:02:11 2013 -0800 + Merge pull request #226 from ash211/patch-4 + [Typo: applicaton] + + d6e5473 Wed Dec 4 10:28:50 2013 -0800 + Merge pull request #223 from rxin/transient + [Mark partitioner, name, and generator field in RDD as @transient.] + + 8a3475a Tue Dec 3 14:21:40 2013 -0800 + Merge pull request #218 from JoshRosen/spark-970-pyspark-unicode-error + [Fix UnicodeEncodeError in PySpark saveAsTextFile() (SPARK-970)] + + 46b87b8 Tue Dec 3 00:41:11 2013 -0800 + Merge pull request #2 from colorant/yarn-client-2.2 + [Fix pom.xml for maven build] + + 58d9bbc Mon Dec 2 21:58:53 2013 -0800 + Merge pull request #217 from aarondav/mesos-urls + [Re-enable zk:// urls for Mesos SparkContexts] + + 740922f Sun Dec 1 12:46:58 2013 -0800 + Merge pull request #219 from sundeepn/schedulerexception + [Scheduler quits when newStage fails] + + 60e23a5 Sat Nov 30 23:38:49 2013 -0800 + Merge pull request #216 from liancheng/fix-spark-966 + [Bugfix: SPARK-965 & SPARK-966] + + 34ee814 Sat Nov 30 15:10:30 2013 -0800 + Merged Ankur's pull request #80 and fixed subgraph. + [] + + 743a31a Wed Nov 27 18:24:39 2013 -0800 + Merge pull request #210 from haitaoyao/http-timeout + [add http timeout for httpbroadcast] + + 993e293 Wed Nov 27 00:57:54 2013 -0800 + Merge pull request #1 from colorant/yarn-client-2.2 + [Port yarn-client mode for new-yarn] + + fb6875d Tue Nov 26 20:55:40 2013 -0800 + Merge pull request #146 from JoshRosen/pyspark-custom-serializers + [Custom Serializers for PySpark] + + 330ada1 Tue Nov 26 19:08:33 2013 -0800 + Merge pull request #207 from henrydavidge/master + [Log a warning if a task's serialized size is very big] + + 615213f Tue Nov 26 19:07:20 2013 -0800 + Merge pull request #212 from markhamstra/SPARK-963 + [[SPARK-963] Fixed races in JobLoggerSuite] + + cb976df Tue Nov 26 10:23:19 2013 -0800 + Merge pull request #209 from pwendell/better-docs + [Improve docs for shuffle instrumentation] + + 18d6df0 Tue Nov 26 00:00:07 2013 -0800 + Merge pull request #86 from holdenk/master + [Add histogram functionality to DoubleRDDFunctions] + + 0e2109d Mon Nov 25 20:48:37 2013 -0800 + Merge pull request #204 from rxin/hash + [OpenHashSet fixes] + + c46067f Mon Nov 25 19:09:31 2013 -0800 + Merge pull request #206 from ash211/patch-2 + [Update tuning.md] + + 14bb465 Mon Nov 25 18:50:18 2013 -0800 + Merge pull request #201 from rxin/mappartitions + [Use the proper partition index in mapPartitionsWIthIndex] + + eb4296c Mon Nov 25 15:25:29 2013 -0800 + Merge pull request #101 from colorant/yarn-client-scheduler + [For SPARK-527, Support spark-shell when running on YARN] + + 466fd06 Mon Nov 25 18:27:26 2013 +0800 + Incorporated ideas from pull request #200. - Use Murmur Hash 3 finalization step to scramble the bits of HashCode instead of the simpler version in java.util.HashMap; the latter one had trouble with ranges of consecutive integers. Murmur Hash 3 is used by fastutil. + [- Don't check keys for equality when re-inserting due to growing the] + + 088995f Mon Nov 25 00:57:51 2013 -0800 + Merge pull request #77 from amplab/upgrade + [Sync with Spark master] + + 62889c4 Mon Nov 25 11:27:45 2013 +0800 + Merge pull request #203 from witgo/master + [ Fix Maven build for metrics-graphite] + + 6af03ed Sun Nov 24 16:42:37 2013 -0800 + Merge pull request #76 from dcrankshaw/fix_partitioners + [Actually use partitioner command line args in Analytics.] + + 859d62d Sun Nov 24 16:19:51 2013 -0800 + Merge pull request #151 from russellcardullo/add-graphite-sink + [Add graphite sink for metrics] + + 65de73c Sun Nov 24 15:52:33 2013 -0800 + Merge pull request #185 from mkolod/random-number-generator + [XORShift RNG with unit tests and benchmark] + + 972171b Mon Nov 25 07:50:46 2013 +0800 + Merge pull request #197 from aarondav/patrick-fix + [Fix 'timeWriting' stat for shuffle files] + + a1a7e36 Sun Nov 24 05:15:09 2013 -0800 + Merge pull request #75 from amplab/simplify + [Simplify GraphImpl internals] + + 718cc80 Sun Nov 24 11:02:02 2013 +0800 + Merge pull request #200 from mateiz/hash-fix + [AppendOnlyMap fixes] + + 51aa9d6 Sat Nov 23 19:46:46 2013 +0800 + Merge pull request #198 from ankurdave/zipPartitions-preservesPartitioning + [Support preservesPartitioning in RDD.zipPartitions] + + 18ce7e9 Fri Nov 22 17:02:40 2013 -0800 + Merge pull request #73 from jegonzal/TriangleCount + [Triangle count] + + 086b097 Fri Nov 22 10:26:39 2013 +0800 + Merge pull request #193 from aoiwelle/patch-1 + [Fix Kryo Serializer buffer documentation inconsistency] + + f20093c Fri Nov 22 10:12:13 2013 +0800 + Merge pull request #196 from pwendell/master + [TimeTrackingOutputStream should pass on calls to close() and flush().] + + 4b89501 Wed Nov 20 10:36:10 2013 -0800 + Merge pull request #191 from hsaputra/removesemicolonscala + [Cleanup to remove semicolons (;) from Scala code] + + 202f8e6 Wed Nov 20 03:26:08 2013 -0800 + Merge pull request #74 from dcrankshaw/remove_sleep + [Removed sleep from pagerank in Analytics] + + 74ade9e Tue Nov 19 16:53:58 2013 -0800 + Merge pull request #62 from dcrankshaw/partitioners + [Allow user to choose a partitioner at runtime] + + f568912 Tue Nov 19 16:11:31 2013 -0800 + Merge pull request #181 from BlackNiuza/fix_tasks_number + [correct number of tasks in ExecutorsUI] + + aa638ed Tue Nov 19 16:05:44 2013 -0800 + Merge pull request #189 from tgravescs/sparkYarnErrorHandling + [Impove Spark on Yarn Error handling] + + 5592580 Tue Nov 19 16:04:01 2013 -0800 + Merge pull request #187 from aarondav/example-bcast-test + [Enable the Broadcast examples to work in a cluster setting] + + 99cfe89 Mon Nov 18 22:00:36 2013 -0500 + Updates to reflect pull request code review + [] + + e2ebc3a Sun Nov 17 18:42:18 2013 -0800 + Merge pull request #182 from rxin/vector + [Slightly enhanced PrimitiveVector:] + + 26f616d Sun Nov 17 18:18:16 2013 -0800 + Merge pull request #3 from aarondav/pv-test + [Add PrimitiveVectorSuite and fix bug in resize()] + + 1b5b358 Sat Nov 16 11:44:10 2013 -0800 + Merge pull request #178 from hsaputra/simplecleanupcode + [Simple cleanup on Spark's Scala code] + + 62a2a71 Fri Nov 15 13:12:07 2013 -0800 + Merge pull request #65 from amplab/varenc + [Use variable encoding for ints, longs, and doubles in the specialized serializers.] + + f6b2e59 Thu Nov 14 23:04:55 2013 -0800 + Merge pull request #1 from aarondav/scala210-master + [Various merge corrections] + + 96e0fb4 Thu Nov 14 22:29:28 2013 -0800 + Merge pull request #173 from kayousterhout/scheduler_hang + [Fix bug where scheduler could hang after task failure.] + + dfd40e9 Thu Nov 14 19:44:50 2013 -0800 + Merge pull request #175 from kayousterhout/no_retry_not_serializable + [Don't retry tasks when they fail due to a NotSerializableException] + + ed25105 Thu Nov 14 19:43:55 2013 -0800 + Merge pull request #174 from ahirreddy/master + [Write Spark UI url to driver file on HDFS] + + 1a4cfbe Thu Nov 14 10:32:11 2013 -0800 + Merge pull request #169 from kayousterhout/mesos_fix + [Don't ignore spark.cores.max when using Mesos Coarse mode] + + 5a4f483 Thu Nov 14 10:30:36 2013 -0800 + Merge pull request #170 from liancheng/hadooprdd-doc-typo + [Fixed a scaladoc typo in HadoopRDD.scala] + + d76f520 Thu Nov 14 10:25:48 2013 -0800 + Merge pull request #171 from RIA-pierre-borckmans/master + [Fixed typos in the CDH4 distributions version codes.] + + 2c39d80 Wed Nov 13 23:28:01 2013 -0800 + Merge pull request #69 from jegonzal/MissingVertices + [Addressing issue in Graph creation] + + 33b2dea Wed Nov 13 17:55:58 2013 -0800 + Merge pull request #1 from ankurdave/MissingVertices + [During graph creation, create eTable earlier] + + 2054c61 Wed Nov 13 16:49:55 2013 -0800 + Merge pull request #159 from liancheng/dagscheduler-actor-refine + [Migrate the daemon thread started by DAGScheduler to Akka actor] + + 9290e5b Wed Nov 13 16:48:44 2013 -0800 + Merge pull request #165 from NathanHowell/kerberos-master + [spark-assembly.jar fails to authenticate with YARN ResourceManager] + + a81fcb7 Wed Nov 13 10:41:01 2013 -0800 + Merge pull request #68 from jegonzal/BitSetSetUntilBug + [Addressing bug in BitSet.setUntil(ind)] + + 39af914 Wed Nov 13 08:39:05 2013 -0800 + Merge pull request #166 from ahirreddy/simr-spark-ui + [SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to ...] + + f49ea28 Tue Nov 12 19:13:39 2013 -0800 + Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebase + [Allow spark on yarn to be run from HDFS.] + + 87f2f4e Tue Nov 12 16:26:09 2013 -0800 + Merge pull request #153 from ankurdave/stop-spot-cluster + [Enable stopping and starting a spot cluster] + + b8bf04a Tue Nov 12 16:19:50 2013 -0800 + Merge pull request #160 from xiajunluan/JIRA-923 + [Fix bug JIRA-923] + + dfd1ebc Tue Nov 12 09:10:05 2013 -0800 + Merge pull request #164 from tdas/kafka-fix + [Made block generator thread safe to fix Kafka bug.] + + 2e8d450 Mon Nov 11 17:34:09 2013 -0800 + Merge pull request #63 from jegonzal/VertexSetCleanup + [Cleanup of VertexSetRDD] + + b8e294a Mon Nov 11 16:25:42 2013 -0800 + Merge pull request #61 from ankurdave/pid2vid + [Shuffle replicated vertex attributes efficiently in columnar format] + + 3d7277c Mon Nov 11 15:49:28 2013 -0800 + Merge pull request #55 from ankurdave/aggregateNeighbors-variants + [Specialize mapReduceTriplets for accessing subsets of vertex attributes] + + 23b53ef Mon Nov 11 12:30:02 2013 -0800 + Merge pull request #156 from haoyuan/master + [add tachyon module] + + 1a06f70 Sun Nov 10 10:54:44 2013 -0800 + Merge pull request #60 from amplab/rxin + [Looks good to me.] + + 58d4f6c Sun Nov 10 09:23:56 2013 -0800 + Merge pull request #157 from rxin/kryo + [3 Kryo related changes.] + + 3efc019 Sat Nov 9 17:53:49 2013 -0800 + Merge pull request #147 from JoshRosen/fix-java-api-completeness-checker + [Add spark-tools assembly to spark-class'ss classpath] + + 87954d4 Sat Nov 9 17:53:25 2013 -0800 + Merge pull request #154 from soulmachine/ClusterScheduler + [Replace the thread inside ClusterScheduler.start() with an Akka scheduler] + + f6c9462 Sat Nov 9 16:14:45 2013 -0800 + Merge pull request #58 from jegonzal/KryoMessages + [Kryo messages] + + 83bf192 Sat Nov 9 15:40:29 2013 -0800 + Merge pull request #155 from rxin/jobgroup + [Don't reset job group when a new job description is set.] + + 8af99f2 Sat Nov 9 13:48:00 2013 -0800 + Merge pull request #149 from tgravescs/fixSecureHdfsAccess + [Fix secure hdfs access for spark on yarn] + + 72a601e Sat Nov 9 11:55:16 2013 -0800 + Merge pull request #152 from rxin/repl + [Propagate SparkContext local properties from spark-repl caller thread to the repl execution thread.] + + 6ee05be Thu Nov 7 19:12:41 2013 -0800 + Merge pull request #49 from jegonzal/graphxshell + [GraphX Console with Logo Text] + + a9f96b5 Thu Nov 7 18:56:56 2013 -0800 + Merge pull request #56 from jegonzal/PregelAPIChanges + [Changing Pregel API to use mapReduceTriplets instead of aggregateNeighbors] + + 5907137 Thu Nov 7 16:58:31 2013 -0800 + Merge pull request #54 from amplab/rxin + [Converted for loops to while loops in EdgePartition.] + + edf4164 Thu Nov 7 16:22:43 2013 -0800 + Merge pull request #53 from amplab/rxin + [Added GraphX to classpath.] + + c379e10 Thu Nov 7 16:01:47 2013 -0800 + Merge pull request #51 from jegonzal/VertexSetRDD + [Reverting to Array based (materialized) output in VertexSetRDD] + + 3d4ad84 Thu Nov 7 11:08:27 2013 -0800 + Merge pull request #148 from squito/include_appId + [Include appId in executor cmd line args] + + be7e8da Wed Nov 6 23:22:47 2013 -0800 + Merge pull request #23 from jerryshao/multi-user + [Add Spark multi-user support for standalone mode and Mesos] + + aadeda5 Wed Nov 6 13:27:47 2013 -0800 + Merge pull request #144 from liancheng/runjob-clean + [Removed unused return value in SparkContext.runJob] + + 951024f Wed Nov 6 09:36:14 2013 -0800 + Merge pull request #145 from aarondav/sls-fix + [Attempt to fix SparkListenerSuite breakage] + + bf4e613 Tue Nov 5 23:14:09 2013 -0800 + Merge pull request #143 from rxin/scheduler-hang + [Ignore a task update status if the executor doesn't exist anymore.] + + 9f7b9bb Tue Nov 5 10:42:19 2013 -0800 + Merge pull request #142 from liancheng/dagscheduler-pattern-matching + [Using case class deep match to simplify code in DAGScheduler.processEvent] + + ca44b51 Tue Nov 5 01:32:55 2013 -0800 + Merge pull request #50 from amplab/mergemerge + [Merge Spark master into graphx] + + 8106532 Mon Nov 4 20:47:14 2013 -0800 + Merge pull request #139 from aarondav/shuffle-next + [Never store shuffle blocks in BlockManager] + + 0b26a39 Mon Nov 4 18:22:06 2013 -0800 + Merge pull request #128 from shimingfei/joblogger-doc + [add javadoc to JobLogger, and some small fix] + + 7a26104 Mon Nov 4 17:54:06 2013 -0800 + Merge pull request #130 from aarondav/shuffle + [Memory-optimized shuffle file consolidation] + + b5dc339 Sun Nov 3 20:43:15 2013 -0800 + Merge pull request #70 from rxin/hash1 + [Fast, memory-efficient hash set, hash table implementations optimized for primitive data types.] + + 41ead7a Sat Nov 2 14:41:50 2013 -0700 + Merge pull request #133 from Mistobaan/link_fix + [update default github] + + d407c07 Sat Nov 2 14:36:37 2013 -0700 + Merge pull request #134 from rxin/readme + [Fixed a typo in Hadoop version in README.] + + e7c7b80 Fri Nov 1 17:58:10 2013 -0700 + Merge pull request #132 from Mistobaan/doc_fix + [fix persistent-hdfs] + + d6d11c2 Fri Nov 1 15:40:33 2013 -0700 + Merge pull request #129 from velvia/2013-11/document-local-uris + [Document & finish support for local: URIs] + + 99bfcc9 Thu Oct 31 21:38:10 2013 -0700 + Merge pull request #46 from jegonzal/VertexSetWithHashSet + [Switched VertexSetRDD and GraphImpl to use OpenHashSet] + + fcaaf86 Thu Oct 31 18:27:30 2013 -0700 + Merge pull request #44 from jegonzal/rxinBitSet + [Switching to VertexSetRDD to use @rxin BitSet and OpenHash ] + + 3f3c727 Thu Oct 31 09:52:25 2013 -0700 + Merge pull request #41 from jegonzal/LineageTracking + [Optimizing Graph Lineage] + + 944f6b8 Thu Oct 31 09:40:35 2013 -0700 + Merge pull request #43 from amplab/FixBitSetCastException + [Fix BitSet cast exception] + + 8f1098a Wed Oct 30 20:11:48 2013 -0700 + Merge pull request #117 from stephenh/avoid_concurrent_modification_exception + [Handle ConcurrentModificationExceptions in SparkContext init.] + + dc9ce16 Wed Oct 30 17:01:56 2013 -0700 + Merge pull request #126 from kayousterhout/local_fix + [Fixed incorrect log message in local scheduler] + + 33de11c Wed Oct 30 16:58:27 2013 -0700 + Merge pull request #124 from tgravescs/sparkHadoopUtilFix + [Pull SparkHadoopUtil out of SparkEnv (jira SPARK-886)] + + a0c86c3 Wed Oct 30 15:34:39 2013 -0700 + Merge pull request #38 from jegonzal/Documentation + [Improving Documentation] + + 618c1f6 Wed Oct 30 12:03:44 2013 -0700 + Merge pull request #125 from velvia/2013-10/local-jar-uri + [Add support for local:// URI scheme for addJars()] + + 745dc42 Tue Oct 29 23:47:10 2013 -0700 + Merge pull request #118 from JoshRosen/blockinfo-memory-usage + [Reduce the memory footprint of BlockInfo objects] + + 06adf63 Tue Oct 29 16:43:46 2013 -0700 + Merge pull request #33 from kellrott/master + [Fixing graph/pom.xml] + + 098768e Tue Oct 29 15:08:36 2013 -0700 + Merge pull request #37 from jegonzal/AnalyticsCleanup + [Updated Connected Components and Pregel Docs] + + f0e23a0 Tue Oct 29 01:41:44 2013 -0400 + Merge pull request #119 from soulmachine/master + [A little revise for the document] + + aec9bf9 Sun Oct 27 19:32:00 2013 -0700 + Merge pull request #112 from kayousterhout/ui_task_attempt_id + [Display both task ID and task attempt ID in UI, and rename taskId to taskAttemptId] + + d4df474 Sun Oct 27 22:11:21 2013 -0400 + Merge pull request #115 from aarondav/shuffle-fix + [Eliminate extra memory usage when shuffle file consolidation is disabled] + + e018f2d Sat Oct 26 11:39:15 2013 -0700 + Merge pull request #113 from pwendell/master + [Improve error message when multiple assembly jars are present.] + + 662ee9f Sat Oct 26 11:35:59 2013 -0700 + Merge pull request #114 from soulmachine/master + [A little revise for the document] + + bab496c Fri Oct 25 18:28:43 2013 -0700 + Merge pull request #108 from alig/master + [Changes to enable executing by using HDFS as a synchronization point between driver and executors, as well as ensuring executors exit properly.] + + d307db6 Fri Oct 25 17:26:06 2013 -0700 + Merge pull request #102 from tdas/transform + [Added new Spark Streaming operations] + + 85e2cab Fri Oct 25 14:46:06 2013 -0700 + Merge pull request #111 from kayousterhout/ui_name + [Properly display the name of a stage in the UI.] + + ab35ec4 Fri Oct 25 10:16:18 2013 -0700 + Merge pull request #110 from pwendell/master + [Exclude jopt from kafka dependency.] + + 4f2c943 Thu Oct 24 22:32:02 2013 -0700 + Merge pull request #109 from pwendell/master + [Adding Java/Java Streaming versions of `repartition` with associated tests] + + 99ad4a6 Thu Oct 24 17:08:39 2013 -0700 + Merge pull request #106 from pwendell/master + [Add a `repartition` operator.] + + 5429d62 Thu Oct 24 11:15:55 2013 -0700 + Merge pull request #107 from ScrapCodes/scala-2.10 + [Updating to latest akka 2.2.3, which fixes our only failing test Driver Suite.] + + 6f82c42 Thu Oct 24 11:09:46 2013 -0700 + Merge pull request #34 from jegonzal/AnalyticsCleanup + [Analytics Cleanup] + + 1dc776b Wed Oct 23 22:05:52 2013 -0700 + Merge pull request #93 from kayousterhout/ui_new_state + [Show "GETTING_RESULTS" state in UI.] + + c4b187d Wed Oct 23 21:56:18 2013 -0700 + Merge pull request #105 from pwendell/doc-fix + [Fixing broken links in programming guide] + + a098438 Wed Oct 23 18:03:08 2013 -0700 + Merge pull request #103 from JoshRosen/unpersist-fix + [Add unpersist() to JavaDoubleRDD and JavaPairRDD.] + + dd65964 Wed Oct 23 15:07:59 2013 -0700 + Merge pull request #64 from prabeesh/master + [MQTT Adapter for Spark Streaming] + + 452aa36 Tue Oct 22 23:15:33 2013 -0700 + Merge pull request #97 from ewencp/pyspark-system-properties + [Add classmethod to SparkContext to set system properties.] + + 9dfcf53 Tue Oct 22 16:01:42 2013 -0700 + Merge pull request #100 from JoshRosen/spark-902 + [Remove redundant Java Function call() definitions] + + 49d5cda Tue Oct 22 15:38:02 2013 -0700 + Merge pull request #30 from jegonzal/VertexSetRDD_Tests + [Testing and Documenting VertexSetRDD] + + 97184de Tue Oct 22 13:10:14 2013 -0700 + Merge pull request #99 from pwendell/master + [Use correct formatting for comments in StoragePerfTester] + + c404adb Tue Oct 22 11:30:19 2013 -0700 + Merge pull request #90 from pwendell/master + [SPARK-940: Do not directly pass Stage objects to SparkListener.] + + aa9019f Tue Oct 22 10:30:02 2013 -0700 + Merge pull request #98 from aarondav/docs + [Docs: Fix links to RDD API documentation] + + a0e08f0 Tue Oct 22 10:20:43 2013 -0700 + Merge pull request #82 from JoshRosen/map-output-tracker-refactoring + [Split MapOutputTracker into Master/Worker classes] + + b84193c Mon Oct 21 23:35:13 2013 -0700 + Merge pull request #92 from tgravescs/sparkYarnFixClasspath + [Fix the Worker to use CoarseGrainedExecutorBackend and modify classpath ...] + + 731c94e Mon Oct 21 23:31:38 2013 -0700 + Merge pull request #56 from jerryshao/kafka-0.8-dev + [Upgrade Kafka 0.7.2 to Kafka 0.8.0-beta1 for Spark Streaming] + + 48952d6 Mon Oct 21 22:45:00 2013 -0700 + Merge pull request #87 from aarondav/shuffle-base + [Basic shuffle file consolidation] + + a51359c Mon Oct 21 20:33:29 2013 -0700 + Merge pull request #95 from aarondav/perftest + [Minor: Put StoragePerfTester in org/apache/] + + 39d2e9b Mon Oct 21 18:58:48 2013 -0700 + Merge pull request #94 from aarondav/mesos-fix + [Fix mesos urls] + + aa61bfd Mon Oct 21 11:57:05 2013 -0700 + Merge pull request #88 from rxin/clean + [Made the following traits/interfaces/classes non-public:] + + 35886f3 Sun Oct 20 22:20:32 2013 -0700 + Merge pull request #41 from pwendell/shuffle-benchmark + [Provide Instrumentation for Shuffle Write Performance] + + 5b9380e Sun Oct 20 21:03:51 2013 -0700 + Merge pull request #89 from rxin/executor + [Don't setup the uncaught exception handler in local mode.] + + 261bcf2 Sun Oct 20 17:59:51 2013 -0700 + Merge pull request #80 from rxin/build + [Exclusion rules for Maven build files.] + + edc5e3f Sun Oct 20 17:18:06 2013 -0700 + Merge pull request #75 from JoshRosen/block-manager-cleanup + [Code de-duplication in BlockManager] + + 2a7ae17 Sun Oct 20 11:45:21 2013 -0700 + Merge pull request #84 from rxin/kill1 + [Added documentation for setJobGroup. Also some minor cleanup in SparkContext.] + + e4abb75 Sun Oct 20 09:38:37 2013 -0700 + Merge pull request #85 from rxin/clean + [Moved the top level spark package object from spark to org.apache.spark] + + 136b9b3 Sun Oct 20 02:58:26 2013 -0700 + Basic shuffle file consolidation + [The Spark shuffle phase can produce a large number of files, as one file is created] + + 747f538 Sat Oct 19 23:40:40 2013 -0700 + Merge pull request #83 from ewencp/pyspark-accumulator-add-method + [Add an add() method to pyspark accumulators.] + + 6511bbe Sat Oct 19 11:34:56 2013 -0700 + Merge pull request #78 from mosharaf/master + [Removed BitTorrentBroadcast and TreeBroadcast.] + + f628804 Fri Oct 18 23:19:42 2013 -0700 + Merge pull request #76 from pwendell/master + [Clarify compression property.] + + 599dcb0 Fri Oct 18 22:49:00 2013 -0700 + Merge pull request #74 from rxin/kill + [Job cancellation via job group id.] + + 9cf43cf Fri Oct 18 22:07:21 2013 -0700 + Merge pull request #28 from jegonzal/VertexSetRDD + [Refactoring IndexedRDD to VertexSetRDD.] + + f888a5b Fri Oct 18 22:06:58 2013 -0700 + Merge pull request #29 from ankurdave/unit-tests + [Unit tests for Graph and GraphOps] + + 8de9706 Fri Oct 18 20:32:39 2013 -0700 + Merge pull request #66 from shivaram/sbt-assembly-deps + [Add SBT target to assemble dependencies] + + e5316d0 Fri Oct 18 20:30:56 2013 -0700 + Merge pull request #68 from mosharaf/master + [Faster and stable/reliable broadcast] + + 8d528af Fri Oct 18 20:24:10 2013 -0700 + Merge pull request #71 from aarondav/scdefaults + [Spark shell exits if it cannot create SparkContext] + + 0794bd7 Fri Oct 18 18:59:58 2013 -0700 + Merge pull request #27 from jegonzal/removed_indexedrdd_from_core + [Removing IndexedRDD changes for spark/core] + + 099977f Thu Oct 17 14:17:08 2013 -0700 + Merge pull request #26 from ankurdave/split-vTableReplicated + [Great work!] + + fc26e5b Thu Oct 17 13:21:07 2013 -0700 + Merge pull request #69 from KarthikTunga/master + [Fix for issue SPARK-627. Implementing --config argument in the scripts.] + + cf64f63 Thu Oct 17 11:12:28 2013 -0700 + Merge pull request #67 from kayousterhout/remove_tsl + [Removed TaskSchedulerListener interface.] + + f9973ca Wed Oct 16 15:58:41 2013 -0700 + Merge pull request #65 from tgravescs/fixYarn + [Fix yarn build] + + 28e9c2a Tue Oct 15 23:59:56 2013 -0700 + Merge pull request #63 from pwendell/master + [Fixing spark streaming example and a bug in examples build.] + + 4e46fde Tue Oct 15 23:14:27 2013 -0700 + Merge pull request #62 from harveyfeng/master + [Make TaskContext's stageId publicly accessible.] + + b534606 Tue Oct 15 21:25:03 2013 -0700 + Merge pull request #8 from vchekan/checkpoint-ttl-restore + [Serialize and restore spark.cleaner.ttl to savepoint] + + 6dbd220 Tue Oct 15 19:02:57 2013 -0700 + Merge pull request #34 from kayousterhout/rename + [Renamed StandaloneX to CoarseGrainedX.] + + 983b83f Tue Oct 15 19:02:46 2013 -0700 + Merge pull request #61 from kayousterhout/daemon_thread + [Unified daemon thread pools] + + 3249e0e Tue Oct 15 14:12:33 2013 -0700 + Merge pull request #59 from rxin/warning + [Bump up logging level to warning for failed tasks.] + + 678dec6 Tue Oct 15 10:51:46 2013 -0700 + Merge pull request #58 from hsaputra/update-pom-asf + [Update pom.xml to use version 13 of the ASF parent pom] + + e33b183 Mon Oct 14 22:25:47 2013 -0700 + Merge pull request #29 from rxin/kill + [Job killing] + + 3b11f43 Mon Oct 14 14:20:01 2013 -0700 + Merge pull request #57 from aarondav/bid + [Refactor BlockId into an actual type] + + 9979690 Sat Oct 12 21:23:26 2013 -0700 + Merge pull request #52 from harveyfeng/hadoop-closure + [Add an optional closure parameter to HadoopRDD instantiation to use when creating local JobConfs.] + + dca8009 Fri Oct 11 16:08:15 2013 -0700 + Merge pull request #54 from aoiwelle/remove_unused_imports + [Remove unnecessary mutable imports] + + 0e5052b Fri Oct 11 15:45:16 2013 -0700 + Merge pull request #51 from ScrapCodes/scala-2.10 + [Scala 2.10] + + fb25f32 Fri Oct 11 15:44:43 2013 -0700 + Merge pull request #53 from witgo/master + [Add a zookeeper compile dependency to fix build in maven] + + d6ead47 Fri Oct 11 15:43:01 2013 -0700 + Merge pull request #32 from mridulm/master + [Address review comments, move to incubator spark] + + c71499b Thu Oct 10 17:16:42 2013 -0700 + Merge pull request #19 from aarondav/master-zk + [Standalone Scheduler fault tolerance using ZooKeeper] + + 5867a82 Thu Oct 10 14:02:37 2013 -0700 + Merge pull request #19 from dcrankshaw/master + [Merge canonical 2d partitioner and group edges into benchmarks] + + cd08f73 Thu Oct 10 13:55:47 2013 -0700 + Merge pull request #44 from mateiz/fast-map + [A fast and low-memory append-only map for shuffle operations] + + 4b46d51 Thu Oct 10 13:35:36 2013 -0700 + Merge pull request #17 from amplab/product2 + [product 2 change] + + 320418f Wed Oct 9 16:55:30 2013 -0700 + Merge pull request #49 from mateiz/kryo-fix-2 + [Fix Chill serialization of Range objects] + + 215238c Wed Oct 9 16:49:44 2013 -0700 + Merge pull request #50 from kayousterhout/SPARK-908 + [Fix race condition in SparkListenerSuite (fixes SPARK-908).] + + 7827efc Wed Oct 9 15:07:25 2013 -0700 + Merge pull request #46 from mateiz/py-sort-update + [Fix PySpark docs and an overly long line of code after #38] + + 7b3ae04 Wed Oct 9 12:14:19 2013 -0700 + Merge pull request #45 from pwendell/metrics_units + [Use standard abbreviation in metrics description (MBytes -> MB)] + + b4fa11f Wed Oct 9 11:59:47 2013 -0700 + Merge pull request #38 from AndreSchumacher/pyspark_sorting + [SPARK-705: implement sortByKey() in PySpark] + + 19d445d Wed Oct 9 11:08:34 2013 -0700 + Merge pull request #22 from GraceH/metrics-naming + [SPARK-900 Use coarser grained naming for metrics] + + 7d50f9f Wed Oct 9 10:32:42 2013 -0700 + Merge pull request #35 from MartinWeindel/scala-2.10 + [Fixing inconsistencies and warnings on Scala 2.10 branch] + + 3218fa7 Tue Oct 8 23:44:55 2013 -0700 + Merge pull request #4 from MLnick/implicit-als + [Adding algorithm for implicit feedback data to ALS] + + e67d5b9 Tue Oct 8 22:57:38 2013 -0700 + Merge pull request #43 from mateiz/kryo-fix + [Don't allocate Kryo buffers unless needed] + + ea34c52 Mon Oct 7 20:45:58 2013 -0700 + Merge pull request #42 from pwendell/shuffle-read-perf + [Fix inconsistent and incorrect log messages in shuffle read path] + + 02f37ee Mon Oct 7 15:48:52 2013 -0700 + Merge pull request #39 from pwendell/master + [Adding Shark 0.7.1 to EC2 scripts] + + 213b70a Mon Oct 7 10:54:22 2013 -0700 + Merge pull request #31 from sundeepn/branch-0.8 + [Resolving package conflicts with hadoop 0.23.9] + + d585613 Sat Oct 5 22:57:05 2013 -0700 + Merge pull request #37 from pwendell/merge-0.8 + [merge in remaining changes from `branch-0.8`] + + 4a25b11 Sat Oct 5 19:28:55 2013 -0700 + Merge pull request #20 from harveyfeng/hadoop-config-cache + [Allow users to pass broadcasted Configurations and cache InputFormats across Hadoop file reads.] + + 8fc68d0 Sat Oct 5 17:24:35 2013 -0700 + Merge pull request #36 from pwendell/versions + [Bumping EC2 default version in master to .] + + 100222b Sat Oct 5 13:38:59 2013 -0700 + Merge pull request #27 from davidmccauley/master + [SPARK-920/921 - JSON endpoint updates] + + 0864193 Sat Oct 5 13:25:18 2013 -0700 + Merge pull request #33 from AndreSchumacher/pyspark_partition_key_change + [Fixing SPARK-602: PythonPartitioner] + + 61ffcde Fri Oct 4 10:52:17 2013 -0700 + Merge pull request #15 from dcrankshaw/master + [Add synthetic generators] + + 3fe12cc Fri Oct 4 10:51:28 2013 -0700 + Merge pull request #946 from ScrapCodes/scala-2.10 + [Fixed non termination of Executor backend, when sc.stop is not called and system.exit instead.] + + 232765f Thu Oct 3 12:00:48 2013 -0700 + Merge pull request #26 from Du-Li/master + [fixed a wildcard bug in make-distribution.sh; ask sbt to check local] + + 405e69b Thu Oct 3 10:52:41 2013 -0700 + Merge pull request #25 from CruncherBigData/master + [Update README: updated the link] + + 49dbfcc Thu Oct 3 10:52:06 2013 -0700 + Merge pull request #28 from tgravescs/sparYarnAppName + [Allow users to set the application name for Spark on Yarn] + + e597ea3 Wed Oct 2 21:14:24 2013 -0700 + Merge pull request #10 from kayousterhout/results_through-bm + [Send Task results through the block manager when larger than Akka frame size (fixes SPARK-669).] + + 714fdab Thu Sep 26 14:28:55 2013 -0700 + Merge pull request #17 from rxin/optimize + [Remove -optimize flag] + + 13eced7 Thu Sep 26 14:18:19 2013 -0700 + Merge pull request #16 from pwendell/master + [Bug fix in master build] + + 70a0b99 Thu Sep 26 14:11:54 2013 -0700 + Merge pull request #14 from kayousterhout/untangle_scheduler + [Improved organization of scheduling packages.] + + afd03b2 Thu Sep 26 14:09:55 2013 -0700 + Merge pull request #943 from ScrapCodes/scala-2.10 + [Scala 2.10 with akka 2.2] + + 76677b8 Thu Sep 26 14:03:46 2013 -0700 + Merge pull request #670 from jey/ec2-ssh-improvements + [EC2 SSH improvements] + + c514cd1 Thu Sep 26 13:48:20 2013 -0700 + Merge pull request #930 from holdenk/master + [Add mapPartitionsWithIndex] + + 560ee5c Thu Sep 26 11:27:34 2013 -0700 + Merge pull request #7 from wannabeast/memorystore-fixes + [some minor fixes to MemoryStore] + + 6566a19 Thu Sep 26 08:01:04 2013 -0700 + Merge pull request #9 from rxin/limit + [Smarter take/limit implementation.] + + 834686b Sun Sep 22 15:06:48 2013 -0700 + Merge pull request #928 from jerryshao/fairscheduler-refactor + [Refactor FairSchedulableBuilder] + + a2ea069 Sat Sep 21 23:04:42 2013 -0700 + Merge pull request #937 from jerryshao/localProperties-fix + [Fix PR926 local properties issues in Spark Streaming like scenarios] + + f06f2da Sat Sep 21 22:43:34 2013 -0700 + Merge pull request #941 from ilikerps/master + [Add "org.apache." prefix to packages in spark-class] + + 7bb12a2 Sat Sep 21 22:42:46 2013 -0700 + Merge pull request #940 from ankurdave/clear-port-properties-after-tests + [After unit tests, clear port properties unconditionally] + + a00317b Fri Sep 20 11:29:31 2013 -0700 + Merge pull request #1 from ankurdave/aggregateNeighbors-returns-graph + [Return Graph from Graph.aggregateNeighbors] + + 6a5e665 Thu Sep 19 22:41:44 2013 -0700 + Merge pull request #3 from ankurdave/clear-port-properties-after-tests + [After unit tests, clear port properties unconditionally ] + + 68ad33a Thu Sep 19 21:30:27 2013 -0700 + Merge pull request #2 from ankurdave/package-fixes + [Package fixes (spark.graph -> org.apache.spark.graph)] + + cd7222c Thu Sep 19 14:21:24 2013 -0700 + Merge pull request #938 from ilikerps/master + [Fix issue with spark_ec2 seeing empty security groups] + + e0dd24d Sat Aug 31 17:54:15 2013 -0700 + Merge pull request #879 from AndreSchumacher/scala-2.10 + [PySpark: replacing class manifest by class tag for Scala 2.10.2 in rdd.py] + + ad61349 Thu Jul 18 13:53:48 2013 -0700 + Merge pull request #709 from ScrapCodes/scala-2.10 + [Fixed warnings in scala 2.10 branch.] + + a289ded Mon Jul 15 15:59:43 2013 -0700 + Merge pull request #700 from ScrapCodes/scala-2.10 + [Scala 2.10 ] + + 1044a95 Fri Jun 14 20:04:24 2013 -0700 + Merge pull request #652 from ScrapCodes/scala-2.10 + [Fixed maven build without netty fix] + + 4b57f83 Sat Apr 20 10:40:07 2013 -0700 + Merge pull request #535 from ScrapCodes/scala-2.10-repl-port + [porting of repl to scala-2.10] + + 73b3fee Sun Jan 20 10:11:49 2013 -0800 + Merge pull request #388 from folone/master + [Updated maven build configuration for Scala 2.10] + + 20adf27 Tue Jan 15 11:03:49 2013 -0800 + Merge pull request #371 from folone/master + [Scala 2.10.0] + +Release 0.8.0-incubating + + 2aff798 Sun Sep 15 14:05:04 2013 -0700 + Merge pull request #933 from jey/yarn-typo-fix + [Fix typo in Maven build docs] + + dbd2c4f Sun Sep 15 13:20:41 2013 -0700 + Merge pull request #932 from pwendell/mesos-version + [Bumping Mesos version to 0.13.0] + + 9fb0b9d Sun Sep 15 13:02:53 2013 -0700 + Merge pull request #931 from pwendell/yarn-docs + [Explain yarn.version in Maven build docs] + + c4c1db2 Fri Sep 13 19:52:12 2013 -0700 + Merge pull request #929 from pwendell/master + [Use different Hadoop version for YARN artifacts.] + + a310de6 Wed Sep 11 19:36:11 2013 -0700 + Merge pull request #926 from kayousterhout/dynamic + [Changed localProperties to use ThreadLocal (not DynamicVariable).] + + 58c7d8b Wed Sep 11 17:33:42 2013 -0700 + Merge pull request #927 from benh/mesos-docs + [Updated Spark on Mesos documentation.] + + 91a59e6 Wed Sep 11 10:21:48 2013 -0700 + Merge pull request #919 from mateiz/jets3t + [Add explicit jets3t dependency, which is excluded in hadoop-client] + + b9128d3 Wed Sep 11 10:03:06 2013 -0700 + Merge pull request #922 from pwendell/port-change + [Change default port number from 3030 to 4030.] + + e07eef8 Wed Sep 11 07:35:39 2013 -0700 + Merge pull request #925 from davidmccauley/master + [SPARK-894 - Not all WebUI fields delivered VIA JSON] + + 8432f27 Tue Sep 10 23:19:53 2013 -0700 + Merge pull request #923 from haoyuan/master + [fix run-example script] + + d40f140 Tue Sep 10 23:05:29 2013 -0700 + Merge pull request #921 from pwendell/master + [Fix HDFS access bug with assembly build.] + + 0a6c051 Mon Sep 9 23:37:57 2013 -0700 + Merge pull request #918 from pwendell/branch-0.8 + [Update versions for 0.8.0 release.] + + 8c14f4b Mon Sep 9 22:07:58 2013 -0700 + Merge pull request #917 from pwendell/master + [Document libgfortran dependency for MLBase] + + c81377b Mon Sep 9 20:16:19 2013 -0700 + Merge pull request #915 from ooyala/master + [Get rid of / improve ugly NPE when Utils.deleteRecursively() fails] + + 61d2a01 Mon Sep 9 18:21:01 2013 -0700 + Merge pull request #916 from mateiz/mkdist-fix + [Fix copy issue in https://github.com/mesos/spark/pull/899] + + a85758c Mon Sep 9 13:45:40 2013 -0700 + Merge pull request #907 from stephenh/document_coalesce_shuffle + [Add better docs for coalesce.] + + 084fc36 Mon Sep 9 12:01:35 2013 -0700 + Merge pull request #912 from tgravescs/ganglia-pom + [Add metrics-ganglia to core pom file] + + 0456384 Mon Sep 9 09:57:54 2013 -0700 + Merge pull request #911 from pwendell/ganglia-sink + [Adding Manen dependency for Ganglia] + + bf984e2 Sun Sep 8 23:50:24 2013 -0700 + Merge pull request #890 from mridulm/master + [Fix hash bug] + + e9d4f44 Sun Sep 8 23:36:48 2013 -0700 + Merge pull request #909 from mateiz/exec-id-fix + [Fix an instance where full standalone mode executor IDs were passed to] + + 2447b1c Sun Sep 8 22:27:49 2013 -0700 + Merge pull request #910 from mateiz/ml-doc-tweaks + [Small tweaks to MLlib docs] + + 7d3204b Sun Sep 8 21:39:12 2013 -0700 + Merge pull request #905 from mateiz/docs2 + [Job scheduling and cluster mode docs] + + f1f8371 Sun Sep 8 21:26:11 2013 -0700 + Merge pull request #896 from atalwalkar/master + [updated content] + + f68848d Sun Sep 8 18:32:16 2013 -0700 + Merge pull request #906 from pwendell/ganglia-sink + [Clean-up of Metrics Code/Docs and Add Ganglia Sink] + + 0b95799 Sun Sep 8 15:30:16 2013 -0700 + Merge pull request #908 from pwendell/master + [Fix target JVM version in scala build] + + 04cfb3a Sun Sep 8 10:33:20 2013 -0700 + Merge pull request #898 from ilikerps/660 + [SPARK-660: Add StorageLevel support in Python] + + 38488ac Sun Sep 8 00:28:53 2013 -0700 + Merge pull request #900 from pwendell/cdh-docs + [Provide docs to describe running on CDH/HDP cluster.] + + a8e376e Sat Sep 7 21:16:01 2013 -0700 + Merge pull request #904 from pwendell/master + [Adding Apache license to two files] + + cfde85e Sat Sep 7 13:53:08 2013 -0700 + Merge pull request #901 from ooyala/2013-09/0.8-doc-changes + [0.8 Doc changes for make-distribution.sh] + + 4a7813a Sat Sep 7 13:52:24 2013 -0700 + Merge pull request #903 from rxin/resulttask + [Fixed the bug that ResultTask was not properly deserializing outputId.] + + afe46ba Sat Sep 7 07:28:51 2013 -0700 + Merge pull request #892 from jey/fix-yarn-assembly + [YARN build fixes] + + 2eebeff Fri Sep 6 15:25:22 2013 -0700 + Merge pull request #897 from pwendell/master + [Docs describing Spark monitoring and instrumentation] + + ddcb9d3 Thu Sep 5 23:54:09 2013 -0700 + Merge pull request #895 from ilikerps/821 + [SPARK-821: Don't cache results when action run locally on driver] + + 699c331 Thu Sep 5 20:21:53 2013 -0700 + Merge pull request #891 from xiajunluan/SPARK-864 + [[SPARK-864]DAGScheduler Exception if we delete Worker and StandaloneExecutorBackend then add Worker] + + 5c7494d Wed Sep 4 22:47:03 2013 -0700 + Merge pull request #893 from ilikerps/master + [SPARK-884: Add unit test to validate Spark JSON output] + + a547866 Wed Sep 4 21:11:56 2013 -0700 + Merge pull request #894 from c0s/master + [Updating assembly README to reflect recent changes in the build.] + + 19f7027 Tue Sep 3 14:29:10 2013 -0700 + Merge pull request #878 from tgravescs/yarnUILink + [Link the Spark UI up to the Yarn UI ] + + 68df246 Tue Sep 3 13:01:17 2013 -0700 + Merge pull request #889 from alig/master + [Return the port the WebUI is bound to (useful if port 0 was used)] + + d3dd48f Mon Sep 2 16:44:54 2013 -0700 + Merge pull request #887 from mateiz/misc-fixes + [Miscellaneous fixes for 0.8] + + 636fc0c Mon Sep 2 11:20:39 2013 -0700 + Merge pull request #886 from mateiz/codec + [Fix spark.io.compression.codec and change default codec to LZF] + + d9a53b9 Sun Sep 1 22:12:30 2013 -0700 + Merge pull request #885 from mateiz/win-py + [Allow PySpark to run on Windows] + + 3c520fe Sun Sep 1 17:26:55 2013 -0700 + Merge pull request #884 from mateiz/win-fixes + [Run script fixes for Windows after package & assembly change] + + f957c26 Sun Sep 1 14:53:57 2013 -0700 + Merge pull request #882 from mateiz/package-rename + [Rename spark package to org.apache.spark] + + a30fac1 Sun Sep 1 12:27:50 2013 -0700 + Merge pull request #883 from alig/master + [Don't require the spark home environment variable to be set for standalone mode (change needed by SIMR)] + + 03cc765 Sun Sep 1 10:20:56 2013 -0700 + Merge pull request #881 from pwendell/master + [Extend QuickStart to include next steps] + + 0e9565a Sat Aug 31 18:55:41 2013 -0700 + Merge pull request #880 from mateiz/ui-tweaks + [Various UI tweaks] + + 2b29a1d Sat Aug 31 17:49:45 2013 -0700 + Merge pull request #877 from mateiz/docs + [Doc improvements for 0.8] + + 6edef9c Sat Aug 31 13:39:24 2013 -0700 + Merge pull request #861 from AndreSchumacher/pyspark_sampling_function + [Pyspark sampling function] + + fd89835 Sat Aug 31 13:18:12 2013 -0700 + Merge pull request #870 from JoshRosen/spark-885 + [Don't send SIGINT / ctrl-c to Py4J gateway subprocess] + + 618f0ec Fri Aug 30 18:17:13 2013 -0700 + Merge pull request #869 from AndreSchumacher/subtract + [PySpark: implementing subtractByKey(), subtract() and keyBy()] + + 94bb7fd Fri Aug 30 12:05:13 2013 -0700 + Merge pull request #876 from mbautin/master_hadoop_rdd_conf + [Make HadoopRDD's configuration accessible] + + 9e17e45 Fri Aug 30 00:22:53 2013 -0700 + Merge pull request #875 from shivaram/build-fix + [Fix broken build by removing addIntercept] + + 016787d Thu Aug 29 22:15:14 2013 -0700 + Merge pull request #863 from shivaram/etrain-ridge + [Adding linear regression and refactoring Ridge regression to use SGD] + + 852d810 Thu Aug 29 22:13:15 2013 -0700 + Merge pull request #819 from shivaram/sgd-cleanup + [Change SVM to use {0,1} labels] + + ca71620 Thu Aug 29 21:51:14 2013 -0700 + Merge pull request #857 from mateiz/assembly + [Change build and run instructions to use assemblies] + + 1528776 Thu Aug 29 21:30:47 2013 -0700 + Merge pull request #874 from jerryshao/fix-report-bug + [Fix removed block zero size log reporting] + + abdbacf Wed Aug 28 21:11:31 2013 -0700 + Merge pull request #871 from pwendell/expose-local + [Expose `isLocal` in SparkContext.] + + afcade3 Wed Aug 28 20:15:40 2013 -0700 + Merge pull request #873 from pwendell/master + [Hot fix for command runner] + + baa84e7 Wed Aug 28 12:44:46 2013 -0700 + Merge pull request #865 from tgravescs/fixtmpdir + [Spark on Yarn should use yarn approved directories for spark.local.dir and tmp] + + cd043cf Tue Aug 27 19:50:32 2013 -0700 + Merge pull request #867 from tgravescs/yarnenvconfigs + [Spark on Yarn allow users to specify environment variables ] + + 898da7e Mon Aug 26 20:40:49 2013 -0700 + Merge pull request #859 from ianbuss/sbt_opts + [Pass SBT_OPTS environment through to sbt_launcher] + + 17bafea Mon Aug 26 11:59:32 2013 -0700 + Merge pull request #864 from rxin/json1 + [Revert json library change] + + f9fc5c1 Sat Aug 24 15:19:56 2013 -0700 + Merge pull request #603 from pwendell/ec2-updates + [Several Improvements to EC2 Scripts] + + d282c1e Fri Aug 23 11:20:20 2013 -0700 + Merge pull request #860 from jey/sbt-ide-fixes + [Fix IDE project generation under SBT] + + 5a6ac12 Thu Aug 22 22:08:03 2013 -0700 + Merge pull request #701 from ScrapCodes/documentation-suggestions + [Documentation suggestions for spark streaming.] + + 46ea0c1 Thu Aug 22 15:57:28 2013 -0700 + Merge pull request #814 from holdenk/master + [Create less instances of the random class during ALS initialization.] + + 9ac3d62 Thu Aug 22 15:51:10 2013 -0700 + Merge pull request #856 from jey/sbt-fix-hadoop-0.23.9 + [Re-add removed dependency to fix build under Hadoop 0.23.9] + + ae8ba83 Thu Aug 22 10:14:54 2013 -0700 + Merge pull request #855 from jey/update-build-docs + [Update build docs] + + 8a36fd0 Thu Aug 22 10:13:35 2013 -0700 + Merge pull request #854 from markhamstra/pomUpdate + [Synced sbt and maven builds to use the same dependencies, etc.] + + c2d00f1 Thu Aug 22 10:13:03 2013 -0700 + Merge pull request #832 from alig/coalesce + [Coalesced RDD with locality] + + e6d66c8 Wed Aug 21 17:44:31 2013 -0700 + Merge pull request #853 from AndreSchumacher/double_rdd + [Implementing SPARK-838: Add DoubleRDDFunctions methods to PySpark] + + 2905611 Tue Aug 20 17:36:14 2013 -0700 + Merge pull request #851 from markhamstra/MutablePairTE + [Removed meaningless types] + + d61337f Tue Aug 20 10:06:06 2013 -0700 + Merge pull request #844 from markhamstra/priorityRename + [Renamed 'priority' to 'jobId' and assorted minor changes] + + 8cae72e Mon Aug 19 23:40:04 2013 -0700 + Merge pull request #828 from mateiz/sched-improvements + [Scheduler fixes and improvements] + + efeb142 Mon Aug 19 19:23:50 2013 -0700 + Merge pull request #849 from mateiz/web-fixes + [Small fixes to web UI] + + abdc1f8 Mon Aug 19 18:30:56 2013 -0700 + Merge pull request #847 from rxin/rdd + [Allow subclasses of Product2 in all key-value related classes] + + 8fa0747 Sun Aug 18 17:02:54 2013 -0700 + Merge pull request #840 from AndreSchumacher/zipegg + [Implementing SPARK-878 for PySpark: adding zip and egg files to context ...] + + 1e137a5 Sat Aug 17 22:22:32 2013 -0700 + Merge pull request #846 from rxin/rdd + [Two minor RDD refactoring] + + e89ffc7 Fri Aug 16 14:02:34 2013 -0700 + Merge pull request #839 from jegonzal/zip_partitions + [Currying RDD.zipPartitions ] + + 1fb1b09 Thu Aug 15 22:15:05 2013 -0700 + Merge pull request #841 from rxin/json + [Use the JSON formatter from Scala library and removed dependency on lift-json.] + + c69c489 Thu Aug 15 20:55:09 2013 -0700 + Merge pull request #843 from Reinvigorate/bug-879 + [fixing typo in conf/slaves] + + 230ab27 Thu Aug 15 17:45:17 2013 -0700 + Merge pull request #834 from Daemoen/master + [Updated json output to allow for display of worker state] + + 659553b Thu Aug 15 16:56:31 2013 -0700 + Merge pull request #836 from pwendell/rename + [Rename `memoryBytesToString` and `memoryMegabytesToString`] + + 28369ff Thu Aug 15 16:44:02 2013 -0700 + Merge pull request #829 from JoshRosen/pyspark-unit-tests-python-2.6 + [Fix PySpark unit tests on Python 2.6] + + 1a13460 Thu Aug 15 15:50:44 2013 -0700 + Merge pull request #833 from rxin/ui + [Various UI improvements.] + + 044a088 Wed Aug 14 20:43:49 2013 -0700 + Merge pull request #831 from rxin/scheduler + [A few small scheduler / job description changes.] + + 839f2d4 Wed Aug 14 16:17:23 2013 -0700 + Merge pull request #822 from pwendell/ui-features + [Adding GC Stats to TaskMetrics (and three small fixes)] + + 63446f9 Wed Aug 14 00:17:07 2013 -0700 + Merge pull request #826 from kayousterhout/ui_fix + [Fixed 2 bugs in executor UI (incl. SPARK-877)] + + 3f14cba Tue Aug 13 20:09:51 2013 -0700 + Merge pull request #825 from shivaram/maven-repl-fix + [Set SPARK_CLASSPATH for maven repl tests] + + 596adc6 Tue Aug 13 19:41:34 2013 -0700 + Merge pull request #824 from mateiz/mesos-0.12.1 + [Update to Mesos 0.12.1] + + d316af9 Tue Aug 13 15:31:01 2013 -0700 + Merge pull request #821 from pwendell/print-launch-command + [Print run command to stderr rather than stdout] + + 1f79d21 Tue Aug 13 15:23:54 2013 -0700 + Merge pull request #818 from kayousterhout/killed_fix + [Properly account for killed tasks.] + + 622f83c Tue Aug 13 09:58:52 2013 -0700 + Merge pull request #817 from pwendell/pr_784 + [Minor clean-up in metrics servlet code] + + a0133bf Tue Aug 13 09:28:18 2013 -0700 + Merge pull request #784 from jerryshao/dev-metrics-servlet + [Add MetricsServlet for Spark metrics system] + + e2fdac6 Mon Aug 12 21:26:59 2013 -0700 + Merge pull request #802 from stayhf/SPARK-760-Python + [Simple PageRank algorithm implementation in Python for SPARK-760] + + d3525ba Mon Aug 12 21:02:39 2013 -0700 + Merge pull request #813 from AndreSchumacher/add_files_pyspark + [Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark] + + 9e02da2 Mon Aug 12 20:22:27 2013 -0700 + Merge pull request #812 from shivaram/maven-mllib-tests + [Create SparkContext in beforeAll for MLLib tests] + + 65d0d91 Mon Aug 12 19:00:57 2013 -0700 + Merge pull request #807 from JoshRosen/guava-optional + [Change scala.Option to Guava Optional in Java APIs] + + 4346f0a Mon Aug 12 12:12:12 2013 -0700 + Merge pull request #809 from shivaram/sgd-cleanup + [Clean up scaladoc in ML Lib.] + + ea1b4ba Mon Aug 12 08:09:58 2013 -0700 + Merge pull request #806 from apivovarov/yarn-205 + [Changed yarn.version to 2.0.5 in pom.xml] + + 2a39d2c Sun Aug 11 20:35:09 2013 -0700 + Merge pull request #810 from pwendell/dead_doc_code + [Remove now dead code inside of docs] + + e5b9ed2 Sun Aug 11 17:22:47 2013 -0700 + Merge pull request #808 from pwendell/ui_compressed_bytes + [Report compressed bytes read when calculating TaskMetrics] + + 3796486 Sun Aug 11 14:51:47 2013 -0700 + Merge pull request #805 from woggle/hadoop-rdd-jobconf + [Use new Configuration() instead of slower new JobConf() in SerializableWritable] + + ff9ebfa Sun Aug 11 10:52:55 2013 -0700 + Merge pull request #762 from shivaram/sgd-cleanup + [Refactor SGD options into a new class.] + + 95c62ca Sun Aug 11 10:30:52 2013 -0700 + Merge pull request #804 from apivovarov/master + [Fixed path to JavaALS.java and JavaKMeans.java, fixed hadoop2-yarn profi...] + + 06e4f2a Sat Aug 10 18:06:23 2013 -0700 + Merge pull request #789 from MLnick/master + [Adding Scala version of PageRank example] + + 71c63de Sat Aug 10 10:21:20 2013 -0700 + Merge pull request #795 from mridulm/master + [Fix bug reported in PR 791 : a race condition in ConnectionManager and Connection] + + d17eeb9 Sat Aug 10 09:02:27 2013 -0700 + Merge pull request #785 from anfeng/master + [expose HDFS file system stats via Executor metrics] + + dce5e47 Fri Aug 9 21:53:45 2013 -0700 + Merge pull request #800 from dlyubimov/HBASE_VERSION + [Pull HBASE_VERSION in the head of sbt build] + + cd247ba Fri Aug 9 20:41:13 2013 -0700 + Merge pull request #786 from shivaram/mllib-java + [Java fixes, tests and examples for ALS, KMeans] + + b09d4b7 Fri Aug 9 13:17:08 2013 -0700 + Merge pull request #799 from woggle/sync-fix + [Remove extra synchronization in ResultTask] + + 0bc63bf Fri Aug 9 13:16:25 2013 -0700 + Merge pull request #801 from pwendell/print-launch-command + [Print launch command [Branch 0.8 version]] + + cc6b92e Fri Aug 9 13:00:33 2013 -0700 + Merge pull request #775 from pwendell/print-launch-command + [Log the launch command for Spark daemons] + + f94fc75 Fri Aug 9 10:04:03 2013 -0700 + Merge pull request #788 from shane-huang/sparkjavaopts + [For standalone mode, add worker local env setting of SPARK_JAVA_OPTS as ...] + + 63b6e02 Thu Aug 8 14:02:02 2013 -0700 + Merge pull request #797 from mateiz/chill-0.3.1 + [Update to Chill 0.3.1] + + 9955e5a Thu Aug 8 11:03:38 2013 -0700 + Merge pull request #796 from pwendell/bootstrap-design + [Bootstrap re-design] + + 5133e4b Wed Aug 7 15:50:45 2013 -0700 + Merge pull request #790 from kayousterhout/fix_throughput + [Fixed issue in UI that decreased scheduler throughput by 5x or more] + + 3c8478e Tue Aug 6 23:25:03 2013 -0700 + Merge pull request #747 from mateiz/improved-lr + [Update the Python logistic regression example] + + 6b043a6 Tue Aug 6 22:31:02 2013 -0700 + Merge pull request #724 from dlyubimov/SPARK-826 + [SPARK-826: fold(), reduce(), collect() always attempt to use java serialization] + + de6c4c9 Tue Aug 6 17:09:50 2013 -0700 + Merge pull request #787 from ash211/master + [Update spark-standalone.md] + + df4d10d Tue Aug 6 15:44:05 2013 -0700 + Merge pull request #779 from adatao/adatao-global-SparkEnv + [[HOTFIX] Extend thread safety for SparkEnv.get()] + + d2b0f0c Tue Aug 6 14:49:39 2013 -0700 + Merge pull request #770 from stayhf/SPARK-760-Java + [Simple PageRank algorithm implementation in Java for SPARK-760] + + d031f73 Mon Aug 5 22:33:00 2013 -0700 + Merge pull request #782 from WANdisco/master + [SHARK-94 Log the files computed by HadoopRDD and NewHadoopRDD] + + 1b63dea Mon Aug 5 22:21:26 2013 -0700 + Merge pull request #769 from markhamstra/NegativeCores + [SPARK-847 + SPARK-845: Zombie workers and negative cores] + + 828aff7 Mon Aug 5 21:37:33 2013 -0700 + Merge pull request #776 from gingsmith/master + [adding matrix factorization data generator] + + 8b27789 Mon Aug 5 19:14:52 2013 -0700 + Merge pull request #774 from pwendell/job-description + [Show user-defined job name in UI] + + 550b0cf Mon Aug 5 12:10:32 2013 -0700 + Merge pull request #780 from cybermaster/master + [SPARK-850] + + 22abbc1 Fri Aug 2 16:37:59 2013 -0700 + Merge pull request #772 from karenfeng/ui-843 + [Show app duration] + + 9d7dfd2 Thu Aug 1 17:41:58 2013 -0700 + Merge pull request #743 from pwendell/app-metrics + [Add application metrics to standalone master] + + 6d7afd7 Thu Aug 1 17:13:28 2013 -0700 + Merge pull request #768 from pwendell/pr-695 + [Minor clean-up of fair scheduler UI] + + 5e7b38f Thu Aug 1 14:59:33 2013 -0700 + Merge pull request #695 from xiajunluan/pool_ui + [Enhance job ui in spark ui system with adding pool information] + + 0a96493 Thu Aug 1 11:27:17 2013 -0700 + Merge pull request #760 from karenfeng/heading-update + [Clean up web UI page headers] + + cb7dd86 Thu Aug 1 11:06:10 2013 -0700 + Merge pull request #758 from pwendell/master-json + [Add JSON path to master index page] + + 58756b7 Wed Jul 31 23:45:41 2013 -0700 + Merge pull request #761 from mateiz/kmeans-generator + [Add data generator for K-means] + + ecab635 Wed Jul 31 18:16:55 2013 -0700 + Merge pull request #763 from c0s/assembly + [SPARK-842. Maven assembly is including examples libs and dependencies] + + 39c75f3 Wed Jul 31 15:52:36 2013 -0700 + Merge pull request #757 from BlackNiuza/result_task_generation + [Bug fix: SPARK-837] + + b2b86c2 Wed Jul 31 15:51:39 2013 -0700 + Merge pull request #753 from shivaram/glm-refactor + [Build changes for ML lib] + + 14bf2fe Wed Jul 31 14:18:16 2013 -0700 + Merge pull request #749 from benh/spark-executor-uri + [Added property 'spark.executor.uri' for launching on Mesos.] + + 4ba4c3f Wed Jul 31 13:14:49 2013 -0700 + Merge pull request #759 from mateiz/split-fix + [Use the Char version of split() instead of the String one in MLUtils] + + a386ced Wed Jul 31 11:22:50 2013 -0700 + Merge pull request #754 from rxin/compression + [Compression codec change] + + 0be071a Wed Jul 31 11:11:59 2013 -0700 + Merge pull request #756 from cdshines/patch-1 + [Refactored Vector.apply(length, initializer) replacing excessive code with library method] + + d4556f4 Wed Jul 31 08:48:14 2013 -0700 + Merge pull request #751 from cdshines/master + [Cleaned Partitioner & PythonPartitioner source by taking out non-related logic to Utils] + + 29b8cd3 Tue Jul 30 21:30:33 2013 -0700 + Merge pull request #755 from jerryshao/add-apache-header + [Add Apache license header to metrics system] + + e87de03 Tue Jul 30 15:00:08 2013 -0700 + Merge pull request #744 from karenfeng/bootstrap-update + [Use Bootstrap progress bars in web UI] + + ae57020 Tue Jul 30 14:56:41 2013 -0700 + Merge pull request #752 from rxin/master + [Minor mllib cleanup] + + 8aee118 Tue Jul 30 10:27:54 2013 -0700 + Merge pull request #748 from atalwalkar/master + [made SimpleUpdater consistent with other updaters] + + 468a36c Mon Jul 29 19:44:33 2013 -0700 + Merge pull request #746 from rxin/cleanup + [Internal cleanup] + + 1e1ffb1 Mon Jul 29 19:26:19 2013 -0700 + Merge pull request #745 from shivaram/loss-update-fix + [Remove duplicate loss history in Gradient Descent] + + c99b674 Mon Jul 29 16:32:55 2013 -0700 + Merge pull request #735 from karenfeng/ui-807 + [Totals for shuffle data and CPU time] + + fe7298b Mon Jul 29 14:01:00 2013 -0700 + Merge pull request #741 from pwendell/usability + [Fix two small usability issues] + + c34c0f6 Mon Jul 29 13:18:10 2013 -0700 + Merge pull request #731 from pxinghao/master + [Adding SVM and Lasso] + + f3d72ff Fri Jul 26 17:19:27 2013 -0700 + Merge pull request #739 from markhamstra/toolsPom + [Missing tools/pom.xml scalatest dependency] + + cb36677 Fri Jul 26 16:59:30 2013 -0700 + Merge pull request #738 from harsha2010/pruning + [Fix bug in Partition Pruning.] + + f3cf094 Thu Jul 25 14:53:21 2013 -0700 + Merge pull request #734 from woggle/executor-env2 + [Get more env vars from driver rather than worker] + + 51c2427 Thu Jul 25 00:03:11 2013 -0700 + Merge pull request #732 from ryanlecompte/master + [Refactor Kryo serializer support to use chill/chill-java] + + 52723b9 Wed Jul 24 14:33:02 2013 -0700 + Merge pull request #728 from jey/examples-jar-env + [Fix setting of SPARK_EXAMPLES_JAR] + + 20338c2 Wed Jul 24 14:32:24 2013 -0700 + Merge pull request #729 from karenfeng/ui-811 + [Stage Page updates] + + 5584ebc Wed Jul 24 11:46:46 2013 -0700 + Merge pull request #675 from c0s/assembly + [Building spark assembly for further consumption of the Spark project with a deployed cluster] + + a73f3ee Wed Jul 24 08:59:14 2013 -0700 + Merge pull request #671 from jerryshao/master + [Add metrics system for Spark] + + b011329 Tue Jul 23 22:50:09 2013 -0700 + Merge pull request #727 from rxin/scheduler + [Scheduler code style cleanup.] + + 876125b Tue Jul 23 22:28:21 2013 -0700 + Merge pull request #726 from rxin/spark-826 + [SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure] + + 2f1736c Tue Jul 23 15:53:30 2013 -0700 + Merge pull request #725 from karenfeng/task-start + [Creates task start events] + + 5364f64 Tue Jul 23 13:40:34 2013 -0700 + Merge pull request #723 from rxin/mllib + [Made RegressionModel serializable and added unit tests to make sure predict methods would work.] + + f369e0e Tue Jul 23 13:22:27 2013 -0700 + Merge pull request #720 from ooyala/2013-07/persistent-rdds-api + [Add a public method getCachedRdds to SparkContext] + + 401aac8 Mon Jul 22 16:57:16 2013 -0700 + Merge pull request #719 from karenfeng/ui-808 + [Creates Executors tab for Jobs UI] + + 8ae1436 Mon Jul 22 16:03:04 2013 -0700 + Merge pull request #722 from JoshRosen/spark-825 + [Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev()] + + 15fb394 Sun Jul 21 10:33:38 2013 -0700 + Merge pull request #716 from c0s/webui-port + [Regression: default webui-port can't be set via command line "--webui-port" anymore] + + c40f0f2 Fri Jul 19 13:33:04 2013 -0700 + Merge pull request #711 from shivaram/ml-generators + [Move ML lib data generator files to util/] + + 413b841 Fri Jul 19 13:31:38 2013 -0700 + Merge pull request #717 from viirya/dev1 + [Do not copy local jars given to SparkContext in yarn mode] + + 0d0a47c Thu Jul 18 12:06:37 2013 -0700 + Merge pull request #710 from shivaram/ml-updates + [Updates to LogisticRegression] + + c6235b5 Thu Jul 18 11:43:48 2013 -0700 + Merge pull request #714 from adatao/master + [[BUGFIX] Fix for sbt/sbt script SPARK_HOME setting] + + 009c79e Thu Jul 18 11:41:52 2013 -0700 + Merge pull request #715 from viirya/dev1 + [fix a bug in build process that pulls in two versions of ASM.] + + 985a9e3 Wed Jul 17 22:27:19 2013 -0700 + Merge pull request #712 from stayhf/SPARK-817 + [Consistently invoke bash with /usr/bin/env bash in scripts to make code ...] + + cad48ed Tue Jul 16 21:41:28 2013 -0700 + Merge pull request #708 from ScrapCodes/dependencies-upgrade + [Dependency upgrade Akka 2.0.3 -> 2.0.5] + + 8a8a8f2 Mon Jul 15 23:09:21 2013 -0700 + Merge pull request #705 from rxin/errormessages + [Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.] + + ed8415b Mon Jul 15 16:41:04 2013 -0700 + Merge pull request #703 from karenfeng/ui-802 + [Link to job UI from standalone deploy cluster web UI] + + e3d3e6f Mon Jul 15 14:59:44 2013 -0700 + Merge pull request #702 from karenfeng/ui-fixes + [Adds app name in HTML page titles on job web UI] + + c7877d5 Sun Jul 14 12:58:13 2013 -0700 + Merge pull request #689 from BlackNiuza/application_status + [Bug fix: SPARK-796] + + 10c0593 Sun Jul 14 11:45:18 2013 -0700 + Merge pull request #699 from pwendell/ui-env + [Add `Environment` tab to SparkUI.] + + 89e8549 Sat Jul 13 16:11:08 2013 -0700 + Merge pull request #698 from Reinvigorate/sm-deps-change + [changing com.google.code.findbugs maven coordinates] + + 77c69ae Fri Jul 12 23:05:21 2013 -0700 + Merge pull request #697 from pwendell/block-locations + [Show block locations in Web UI.] + + 5a7835c Fri Jul 12 20:28:21 2013 -0700 + Merge pull request #691 from karenfeng/logpaging + [Create log pages] + + 71ccca0 Fri Jul 12 20:25:06 2013 -0700 + Merge pull request #696 from woggle/executor-env + [Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath.sh] + + 90fc3f3 Fri Jul 12 20:23:36 2013 -0700 + Merge pull request #692 from Reinvigorate/takeOrdered + [adding takeOrdered() to RDD] + + 018d04c Thu Jul 11 12:48:37 2013 -0700 + Merge pull request #684 from woggle/mesos-classloader + [Explicitly set class loader for MesosSchedulerDriver callbacks.] + + bc19477 Wed Jul 10 22:29:41 2013 -0700 + Merge pull request #693 from c0s/readme + [Updating README to reflect Scala 2.9.3 requirements] + + 7dcda9a Mon Jul 8 23:24:23 2013 -0700 + Merge pull request #688 from markhamstra/scalaDependencies + [Fixed SPARK-795 with explicit dependencies] + + 638927b Mon Jul 8 22:58:50 2013 -0700 + Merge pull request #683 from shivaram/sbt-test-fix + [Remove some stack traces from sbt test output] + + 3c13178 Mon Jul 8 14:50:34 2013 -0700 + Merge pull request #687 from atalwalkar/master + [Added "Labeled" to util functions for labeled data] + + 744da8e Sun Jul 7 17:42:25 2013 -0700 + Merge pull request #679 from ryanlecompte/master + [Make binSearch method tail-recursive for RidgeRegression] + + 3cc6818 Sat Jul 6 19:51:20 2013 -0700 + Merge pull request #668 from shimingfei/guava-14.0.1 + [update guava version from 11.0.1 to 14.0.1] + + 2216188 Sat Jul 6 16:18:15 2013 -0700 + Merge pull request #676 from c0s/asf-avro + [Use standard ASF published avro module instead of a proprietory built one] + + 94871e4 Sat Jul 6 15:26:19 2013 -0700 + Merge pull request #655 from tgravescs/master + [Add support for running Spark on Yarn on a secure Hadoop Cluster] + + 3f918b3 Sat Jul 6 12:45:18 2013 -0700 + Merge pull request #672 from holdenk/master + [s/ActorSystemImpl/ExtendedActorSystem/ as ActorSystemImpl results in a warning] + + 2a36e54 Sat Jul 6 12:43:21 2013 -0700 + Merge pull request #673 from xiajunluan/master + [Add config template file for fair scheduler feature] + + 7ba7fa1 Sat Jul 6 11:45:08 2013 -0700 + Merge pull request #674 from liancheng/master + [Bug fix: SPARK-789] + + f4416a1 Sat Jul 6 11:41:58 2013 -0700 + Merge pull request #681 from BlackNiuza/memory_leak + [Remove active job from idToActiveJob when job finished or aborted] + + e063e29 Fri Jul 5 21:54:52 2013 -0700 + Merge pull request #680 from tdas/master + [Fixed major performance bug in Network Receiver] + + bf1311e Fri Jul 5 17:32:44 2013 -0700 + Merge pull request #678 from mateiz/ml-examples + [Start of ML package] + + 6ad85d0 Thu Jul 4 21:32:29 2013 -0700 + Merge pull request #677 from jerryshao/fix_stage_clean + [Clean StageToInfos periodically when spark.cleaner.ttl is enabled] + + 2e32fc8 Thu Jul 4 12:18:20 2013 -0700 + Merge pull request #666 from c0s/master + [hbase dependency is missed in hadoop2-yarn profile of examples module +] + + 6d60fe5 Mon Jul 1 18:24:03 2013 -0700 + Merge pull request #666 from c0s/master + [hbase dependency is missed in hadoop2-yarn profile of examples module] + + ccfe953 Sat Jun 29 17:57:53 2013 -0700 + Merge pull request #577 from skumargithub/master + [Example of cumulative counting using updateStateByKey] + + 50ca176 Thu Jun 27 22:24:52 2013 -0700 + Merge pull request #664 from pwendell/test-fix + [Removing incorrect test statement] + + e49bc8c Wed Jun 26 11:13:33 2013 -0700 + Merge pull request #663 from stephenh/option_and_getenv + [Be cute with Option and getenv.] + + f5e32ed Tue Jun 25 09:16:57 2013 -0700 + Merge pull request #661 from mesos/streaming + [Kafka fixes and DStream.count fix for master] + + 1249e91 Mon Jun 24 21:46:33 2013 -0700 + Merge pull request #572 from Reinvigorate/sm-block-interval + [Adding spark.streaming.blockInterval property] + + cfcda95 Mon Jun 24 21:44:50 2013 -0700 + Merge pull request #571 from Reinvigorate/sm-kafka-serializers + [Surfacing decoders on KafkaInputDStream] + + 575aff6 Mon Jun 24 21:35:50 2013 -0700 + Merge pull request #567 from Reinvigorate/sm-count-fix + [Fixing count() in Spark Streaming] + + 3e61bef Sat Jun 22 16:22:47 2013 -0700 + Merge pull request #648 from shivaram/netty-dbg + [Shuffle fixes and cleanup] + + 1ef5d0d Sat Jun 22 09:35:57 2013 -0700 + Merge pull request #644 from shimingfei/joblogger + [add Joblogger to Spark (on new Spark code)] + + 7e4b266 Sat Jun 22 07:53:18 2013 -0700 + Merge pull request #563 from jey/python-optimization + [Optimize PySpark worker invocation] + + 71030ba Wed Jun 19 15:21:03 2013 -0700 + Merge pull request #654 from lyogavin/enhance_pipe + [fix typo and coding style in #638] + + 73f4c7d Tue Jun 18 04:21:17 2013 -0700 + Merge pull request #605 from esjewett/SPARK-699 + [Add hBase example (retry of pull request #596)] + + 9933836 Tue Jun 18 02:41:10 2013 -0700 + Merge pull request #647 from jerryshao/master + [Reduce ZippedPartitionsRDD's getPreferredLocations complexity from O(2^2n) to O(2^n)] + + db42451 Mon Jun 17 15:26:36 2013 -0700 + Merge pull request #643 from adatao/master + [Bug fix: Zero-length partitions result in NaN for overall mean & variance] + + e82a2ff Mon Jun 17 15:13:15 2013 -0700 + Merge pull request #653 from rxin/logging + [SPARK-781: Log the temp directory path when Spark says "Failed to create temp directory."] + + e6d1277 Mon Jun 17 12:56:25 2013 -0700 + Merge pull request #638 from lyogavin/enhance_pipe + [Enhance pipe to support more features we can do in hadoop streaming] + + f961aac Sat Jun 15 00:53:41 2013 -0700 + Merge pull request #649 from ryanlecompte/master + [Add top K method to RDD using a bounded priority queue] + + 6602d94 Fri Jun 14 10:41:31 2013 -0700 + Merge pull request #651 from rxin/groupbykey + [SPARK-772 / SPARK-774: groupByKey and cogroup should disable map side combine] + + d93851a Thu Jun 13 13:38:45 2013 -0700 + Merge pull request #645 from pwendell/compression + [Adding compression to Hadoop save functions] + + f1da591 Wed Jun 12 17:55:08 2013 -0700 + Merge pull request #646 from markhamstra/jvmArgs + [Fixed jvmArgs in maven build.] + + 0e94b73 Mon Jun 10 13:00:31 2013 -0700 + Merge pull request #625 from stephenh/fix-start-slave + [Fix start-slave not passing instance number to spark-daemon.] + + 74b91d5 Sat Jun 8 01:19:40 2013 -0700 + Merge pull request #629 from c0s/master + [Sometime Maven build runs out of PermGen space.] + + c8fc423 Fri Jun 7 22:43:18 2013 -0700 + Merge pull request #631 from jerryshao/master + [Fix block manager UI display issue when enable spark.cleaner.ttl] + + 1ae60bc Fri Jun 7 22:39:06 2013 -0700 + Merge pull request #634 from xiajunluan/master + [[Spark-753] Fix ClusterSchedulSuite unit test failed ] + + fff3728 Tue Jun 4 16:09:50 2013 -0700 + Merge pull request #640 from pwendell/timeout-update + [Fixing bug in BlockManager timeout] + + f420d4f Tue Jun 4 15:25:58 2013 -0700 + Merge pull request #639 from pwendell/timeout-update + [Bump akka and blockmanager timeouts to 60 seconds] + + 84530ba Fri May 31 17:06:13 2013 -0700 + Merge pull request #636 from rxin/unpersist + [Unpersist More block manager cleanup.] + + ef77bb7 Thu May 30 14:50:06 2013 -0700 + Merge pull request #627 from shivaram/master + [Netty and shuffle bug fixes] + + 8cb8178 Thu May 30 14:17:44 2013 -0700 + Merge pull request #628 from shivaram/zero-block-size + [Skip fetching zero-sized blocks in NIO.] + + 6ed7139 Wed May 29 10:14:22 2013 -0700 + Merge pull request #626 from stephenh/remove-add-if-no-port + [Remove unused addIfNoPort.] + + 41d230c Tue May 28 23:35:24 2013 -0700 + Merge pull request #611 from squito/classloader + [Use default classloaders for akka & deserializing task results] + + 3db1e17 Mon May 27 21:31:43 2013 -0700 + Merge pull request #620 from jerryshao/master + [Fix CheckpointRDD java.io.FileNotFoundException when calling getPreferredLocations] + + 3d4891d Sat May 25 23:38:05 2013 -0700 + Merge pull request #621 from JoshRosen/spark-613 + [Use ec2-metadata in start-slave.sh to detect if running on EC2] + + e8d4b6c Sat May 25 21:09:03 2013 -0700 + Merge pull request #529 from xiajunluan/master + [[SPARK-663]Implement Fair Scheduler in Spark Cluster Scheduler ] + + 9a3c344 Sat May 25 17:53:43 2013 -0700 + Merge pull request #624 from rxin/master + [NonJavaSerializableClass should not be Java serializable...] + + 24e41aa Fri May 24 16:48:52 2013 -0700 + Merge pull request #623 from rxin/master + [Automatically configure Netty port.] + + 69161f9 Fri May 24 14:42:13 2013 -0700 + Merge pull request #622 from rxin/master + [bug fix: Shuffle block iterator is ignoring the shuffle serializer setting.] + + dbbedfc Thu May 23 23:11:06 2013 -0700 + Merge pull request #616 from jey/maven-netty-exclusion + [Exclude old versions of Netty from Maven-based build] + + a2b0a79 Tue May 21 18:16:20 2013 -0700 + Merge pull request #619 from woggling/adjust-sampling + [Use ARRAY_SAMPLE_SIZE constant instead of hard-coded 100.0 in SizeEstimator] + + 66dac44 Tue May 21 11:41:42 2013 -0700 + Merge pull request #618 from woggling/dead-code-disttest + [DistributedSuite: remove dead code] + + 5912cc4 Fri May 17 19:58:40 2013 -0700 + Merge pull request #610 from JoshRosen/spark-747 + [Throw exception if TaskResult exceeds Akka frame size] + + 6c27c38 Thu May 16 17:33:56 2013 -0700 + Merge pull request #615 from rxin/build-fix + [Maven build fix & two other small changes] + + 2f576ab Wed May 15 18:06:24 2013 -0700 + Merge pull request #602 from rxin/shufflemerge + [Manual merge & cleanup of Shane's Shuffle Performance Optimization] + + 48c6f46 Wed May 15 10:47:19 2013 -0700 + Merge pull request #612 from ash211/patch-4 + [Docs: Mention spark shell's default for MASTER] + + 203d7b7 Wed May 15 00:47:20 2013 -0700 + Merge pull request #593 from squito/driver_ui_link + [Master UI has link to Application UI] + + 016ac86 Mon May 13 21:45:36 2013 -0700 + Merge pull request #601 from rxin/emptyrdd-master + [EmptyRDD (master branch 0.8)] + + 4b354e0 Mon May 13 17:39:19 2013 -0700 + Merge pull request #589 from mridulm/master + [Add support for instance local scheduling] + + 5dbc9b2 Sun May 12 11:03:10 2013 -0700 + Merge pull request #608 from pwendell/SPARK-738 + [SPARK-738: Spark should detect and wrap nonserializable exceptions] + + 63e1999 Fri May 10 13:54:03 2013 -0700 + Merge pull request #606 from markhamstra/foreachPartition_fix + [Actually use the cleaned closure in foreachPartition] + + 42bbe89 Wed May 8 22:30:31 2013 -0700 + Merge pull request #599 from JoshRosen/spark-670 + [Fix SPARK-670: EC2 'start' command should require -i option.] + + 0f1b7a0 Wed May 8 13:38:50 2013 -0700 + Merge pull request #596 from esjewett/master + [hBase example] + + 7af92f2 Sat May 4 22:29:17 2013 -0700 + Merge pull request #597 from JoshRosen/webui-fixes + [Two minor bug fixes for Spark Web UI] + + c74ce60 Sat May 4 22:26:35 2013 -0700 + Merge pull request #598 from rxin/blockmanager + [Fixed flaky unpersist test in DistributedSuite.] + + 3bf2c86 Fri May 3 18:27:30 2013 -0700 + Merge pull request #594 from shivaram/master + [Add zip partitions to Java API] + + 2484ad7 Fri May 3 17:08:55 2013 -0700 + Merge pull request #587 from rxin/blockmanager + [A set of shuffle map output related changes] + + 6fe9d4e Thu May 2 21:33:56 2013 -0700 + Merge pull request #592 from woggling/localdir-fix + [Don't accept generated local directory names that can't be created] + + 538ee75 Thu May 2 09:01:42 2013 -0700 + Merge pull request #581 from jerryshao/master + [fix [SPARK-740] block manage UI throws exception when enabling Spark Streaming] + + 9abcbcc Wed May 1 22:45:10 2013 -0700 + Merge pull request #591 from rxin/removerdd + [RDD.unpersist: probably the most desired feature of Spark] + + aa8fe1a Tue Apr 30 22:30:18 2013 -0700 + Merge pull request #586 from mridulm/master + [Pull request to address issues Reynold Xin reported] + + f708dda Tue Apr 30 07:51:40 2013 -0700 + Merge pull request #585 from pwendell/listener-perf + [[Fix SPARK-742] Task Metrics should not employ per-record timing by default] + + 68c07ea Sun Apr 28 20:19:33 2013 -0700 + Merge pull request #582 from shivaram/master + [Add zip partitions interface] + + f6ee9a8 Sun Apr 28 15:36:04 2013 -0700 + Merge pull request #583 from mridulm/master + [Fix issues with streaming test cases after yarn branch merge] + + cf54b82 Thu Apr 25 11:45:58 2013 -0700 + Merge pull request #580 from pwendell/quickstart + [SPARK-739 Have quickstart standlone job use README] + + 118a6c7 Wed Apr 24 08:42:30 2013 -0700 + Merge pull request #575 from mridulm/master + [Manual merge of yarn branch to trunk] + + 5d8a71c Tue Apr 16 19:48:02 2013 -0700 + Merge pull request #570 from jey/increase-codecache-size + [Increase ReservedCodeCacheSize for sbt] + + ec5e553 Sun Apr 14 08:20:13 2013 -0700 + Merge pull request #558 from ash211/patch-jackson-conflict + [Don't pull in old versions of Jackson via hadoop-core] + + c1c219e Sun Apr 14 08:11:23 2013 -0700 + Merge pull request #564 from maspotts/master + [Allow latest scala in PATH, with SCALA_HOME as override (instead of vice-versa)] + + 7c10b3e Fri Apr 12 20:55:22 2013 -0700 + Merge pull request #565 from andyk/master + [Update wording of section on RDD operations in quick start guide in docs] + + 077ae0a Thu Apr 11 19:34:14 2013 -0700 + Merge pull request #561 from ash211/patch-4 + [Add details when BlockManager heartbeats time out] + + c91ff8d Wed Apr 10 15:08:23 2013 -0700 + Merge pull request #560 from ash211/patch-3 + [Typos: cluser -> cluster] + + 7cd83bf Tue Apr 9 22:07:35 2013 -0700 + Merge pull request #559 from ash211/patch-example-whitespace + [Uniform whitespace across scala examples] + + 271a4f3 Tue Apr 9 22:04:52 2013 -0700 + Merge pull request #555 from holdenk/master + [Retry failed ssh commands in the ec2 python script.] + + 8ac9efb Tue Apr 9 13:50:50 2013 -0700 + Merge pull request #527 from Reinvigorate/sm-kafka-cleanup + [KafkaInputDStream fixes and improvements] + + eed54a2 Mon Apr 8 09:44:30 2013 -0700 + Merge pull request #553 from pwendell/akka-standalone + [SPARK-724 - Have Akka logging enabled by default for standalone daemons] + + b362df3 Sun Apr 7 17:17:52 2013 -0700 + Merge pull request #552 from MLnick/master + [Bumping version for Twitter Algebird to latest] + + 4b30190 Sun Apr 7 17:15:10 2013 -0700 + Merge pull request #554 from andyk/scala2.9.3 + [Fixes SPARK-723 - Update build to Scala 2.9.3] + + dfe98ca Tue Apr 2 19:24:12 2013 -0700 + Merge pull request #550 from erikvanoosten/master + [corrected Algebird example] + + b5d7830 Tue Apr 2 19:23:45 2013 -0700 + Merge pull request #551 from jey/python-bugfixes + [Python bugfixes] + + 2be2295 Sun Mar 31 18:09:14 2013 -0700 + Merge pull request #548 from markhamstra/getWritableClass_filter + [Fixed broken filter in getWritableClass[T]] + + 9831bc1 Fri Mar 29 22:16:22 2013 -0700 + Merge pull request #539 from cgrothaus/fix-webui-workdirpath + [Bugfix: WorkerWebUI must respect workDirPath from Worker] + + 3cc8ab6 Fri Mar 29 22:14:07 2013 -0700 + Merge pull request #541 from stephenh/shufflecoalesce + [Add a shuffle parameter to coalesce.] + + cad507a Fri Mar 29 22:13:12 2013 -0700 + Merge pull request #547 from jey/maven-streaming-tests-initialization-fix + [Move streaming test initialization into 'before' blocks] + + a98996d Fri Mar 29 22:12:15 2013 -0700 + Merge pull request #545 from ash211/patch-1 + [Don't use deprecated Application in example] + + 104c694 Fri Mar 29 22:11:50 2013 -0700 + Merge pull request #546 from ash211/patch-2 + [Update tuning.md] + + bc36ee4 Tue Mar 26 15:05:13 2013 -0700 + Merge pull request #543 from holdenk/master + [Re-enable deprecation warnings and fix deprecated warning.] + + b8949ca Sat Mar 23 07:19:34 2013 -0700 + Merge pull request #505 from stephenh/volatile + [Make Executor fields volatile since they're read from the thread pool.] + + fd53f2f Sat Mar 23 07:13:21 2013 -0700 + Merge pull request #510 from markhamstra/WithThing + [mapWith, flatMapWith and filterWith] + + 4c5efcf Wed Mar 20 19:29:23 2013 -0700 + Merge pull request #532 from andyk/master + [SPARK-715: Adds instructions for building with Maven to documentation] + + 3558849 Wed Mar 20 19:27:47 2013 -0700 + Merge pull request #538 from rxin/cogroup + [Added mapSideCombine flag to CoGroupedRDD. Added unit test for CoGroupedRDD.] + + ca4d083 Wed Mar 20 11:22:36 2013 -0700 + Merge pull request #528 from MLnick/java-examples + [[SPARK-707] Adding Java versions of Pi, LogQuery and K-Means examples] + + b812e6b Wed Mar 20 11:21:02 2013 -0700 + Merge pull request #526 from markhamstra/foldByKey + [Add foldByKey] + + 945d1e7 Tue Mar 19 21:59:06 2013 -0700 + Merge pull request #536 from sasurfer/master + [CoalescedRDD for many partitions] + + 1cbbe94 Tue Mar 19 21:34:34 2013 -0700 + Merge pull request #534 from stephenh/removetrycatch + [Remove try/catch block that can't be hit.] + + 71e53f8 Tue Mar 19 21:31:41 2013 -0700 + Merge pull request #537 from wishbear/configurableInputFormat + [call setConf from input format if it is Configurable] + + c1e9cdc Sat Mar 16 11:47:45 2013 -0700 + Merge pull request #525 from stephenh/subtractByKey + [Add PairRDDFunctions.subtractByKey.] + + cdbfd1e Fri Mar 15 15:13:28 2013 -0700 + Merge pull request #516 from squito/fix_local_metrics + [Fix local metrics] + + f9fa2ad Fri Mar 15 15:12:43 2013 -0700 + Merge pull request #530 from mbautin/master-update-log4j-and-make-compile-in-IntelliJ + [Add a log4j compile dependency to fix build in IntelliJ] + + 4032beb Wed Mar 13 19:29:46 2013 -0700 + Merge pull request #521 from stephenh/earlyclose + [Close the reader in HadoopRDD as soon as iteration end.] + + 3c97276 Wed Mar 13 19:25:08 2013 -0700 + Merge pull request #524 from andyk/master + [Fix broken link to YARN documentation] + + 1c3d981 Wed Mar 13 19:23:48 2013 -0700 + Merge pull request #517 from Reinvigorate/sm-build-fixes + [Build fixes for streaming /w SBT] + + 2d477fd Wed Mar 13 06:49:16 2013 -0700 + Merge pull request #523 from andyk/master + [Fix broken link in Quick Start] + + 00c4d23 Tue Mar 12 22:19:00 2013 -0700 + Merge pull request #518 from woggling/long-bm-sizes + [Send block sizes as longs in BlockManager updates] + + cbf8f0d Mon Mar 11 00:23:57 2013 -0700 + Merge pull request #513 from MLnick/bagel-caching + [Adds choice of persistence level to Bagel.] + + 91a9d09 Sun Mar 10 15:48:23 2013 -0700 + Merge pull request #512 from patelh/fix-kryo-serializer + [Fix reference bug in Kryo serializer, add test, update version] + + 557cfd0 Sun Mar 10 15:44:57 2013 -0700 + Merge pull request #515 from woggling/deploy-app-death + [Notify standalone deploy client of application death.] + + 04fb81f Sun Mar 3 17:20:07 2013 -0800 + Merge pull request #506 from rxin/spark-706 + [Fixed SPARK-706: Failures in block manager put leads to read task hanging.] + + 6cf4be4 Sun Mar 3 17:16:22 2013 -0800 + Merge pull request #462 from squito/stageInfo + [Track assorted metrics for each task, report summaries to user at stage completion] + + 6bfc7ca Sat Mar 2 22:14:49 2013 -0800 + Merge pull request #504 from mosharaf/master + [Worker address was getting removed when removing an app.] + + 94b3db1 Sat Mar 2 22:13:52 2013 -0800 + Merge pull request #508 from markhamstra/TestServerInUse + [Avoid bind failure in InputStreamsSuite] + + 25c71d3 Fri Mar 1 08:00:18 2013 -0800 + Merge pull request #507 from markhamstra/poms271 + [bump version to 0.7.1-SNAPSHOT in the subproject poms] + diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py index 2e1a35a629342..916ec90f05844 100755 --- a/dev/create-release/generate-changelist.py +++ b/dev/create-release/generate-changelist.py @@ -31,8 +31,8 @@ import traceback SPARK_HOME = os.environ["SPARK_HOME"] -NEW_RELEASE_VERSION = "1.0.0" -PREV_RELEASE_GIT_TAG = "v0.9.1" +NEW_RELEASE_VERSION = "1.1.0" +PREV_RELEASE_GIT_TAG = "v1.0.0" CHANGELIST = "CHANGES.txt" OLD_CHANGELIST = "%s.old" % (CHANGELIST) From 58b0be6a29eab817d350729710345e9f39e4c506 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 27 Aug 2014 23:28:08 +0000 Subject: [PATCH 140/192] [maven-release-plugin] prepare release v1.1.0-rc1 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index c5a9b3d5f4fd0..b861e77c387f6 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 @@ -881,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From 78e3c036eee7113b2ed144eec5061e070b479e56 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 27 Aug 2014 23:28:27 +0000 Subject: [PATCH 141/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index b861e77c387f6..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From 54ccd93e621c1bc4afc709a208b609232ab701d1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 27 Aug 2014 23:03:46 -0700 Subject: [PATCH 142/192] [HOTFIX] Wait for EOF only for the PySpark shell In `SparkSubmitDriverBootstrapper`, we wait for the parent process to send us an `EOF` before finishing the application. This is applicable for the PySpark shell because we terminate the application the same way. However if we run a python application, for instance, the JVM actually never exits unless it receives a manual EOF from the user. This is causing a few tests to timeout. We only need to do this for the PySpark shell because Spark submit runs as a python subprocess only in this case. Thus, the normal Spark shell doesn't need to go through this case even though it is also a REPL. Thanks davies for reporting this. Author: Andrew Or Closes #2170 from andrewor14/bootstrap-hotfix and squashes the following commits: 42963f5 [Andrew Or] Do not wait for EOF unless this is the pyspark shell (cherry picked from commit dafe343499bbc688e266106e4bb897f9e619834e) Signed-off-by: Patrick Wendell --- bin/pyspark | 2 ++ .../SparkSubmitDriverBootstrapper.scala | 26 +++++++++++-------- 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 01d42025c978e..66876481188aa 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -102,6 +102,8 @@ if [[ "$1" =~ \.py$ ]]; then gatherSparkSubmitOpts "$@" exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" else + # PySpark shell requires special handling downstream + export PYSPARK_SHELL=1 # Only use ipython if no command line arguments were provided [SPARK-1134] if [[ "$IPYTHON" = "1" ]]; then exec ipython $IPYTHON_OPTS diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 7ca96ed57c2db..38b5d8e1739d0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -132,25 +132,29 @@ private[spark] object SparkSubmitDriverBootstrapper { val builder = new ProcessBuilder(filteredCommand) val process = builder.start() - // Redirect stdin, stdout, and stderr to/from the child JVM + // Redirect stdout and stderr from the child JVM val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") stdoutThread.start() stderrThread.start() - // In Windows, the subprocess reads directly from our stdin, so we should avoid spawning - // a thread that contends with the subprocess in reading from System.in. - if (Utils.isWindows) { - // For the PySpark shell, the termination of this process is handled in java_gateway.py - process.waitFor() - } else { - // Terminate on broken pipe, which signals that the parent process has exited. This is - // important for the PySpark shell, where Spark submit itself is a python subprocess. + // Redirect stdin to child JVM only if we're not running Windows. This is because the + // subprocess there already reads directly from our stdin, so we should avoid spawning a + // thread that contends with the subprocess in reading from System.in. + val isWindows = Utils.isWindows + val isPySparkShell = sys.env.contains("PYSPARK_SHELL") + if (!isWindows) { val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") stdinThread.start() - stdinThread.join() - process.destroy() + // For the PySpark shell, Spark submit itself runs as a python subprocess, and so this JVM + // should terminate on broken pipe, which signals that the parent process has exited. In + // Windows, the termination logic for the PySpark shell is handled in java_gateway.py + if (isPySparkShell) { + stdinThread.join() + process.destroy() + } } + process.waitFor() } } From 233c283e3d946bdcbf418375122c5763559c0119 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 27 Aug 2014 23:05:34 -0700 Subject: [PATCH 143/192] [HOTFIX][SQL] Remove cleaning of UDFs It is not safe to run the closure cleaner on slaves. #2153 introduced this which broke all UDF execution on slaves. Will re-add cleaning of UDF closures in a follow-up PR. Author: Michael Armbrust Closes #2174 from marmbrus/fixUdfs and squashes the following commits: 55406de [Michael Armbrust] [HOTFIX] Remove cleaning of UDFs (cherry picked from commit 024178c57419f915d26414e1b91ea0019c3650db) Signed-off-by: Patrick Wendell --- .../org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 0b3c1df453fb8..589816ccec0d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -23,9 +23,6 @@ import org.apache.spark.util.ClosureCleaner case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { - // Clean function when not called with default no-arg constructor. - if (function != null) { ClosureCleaner.clean(function) } - type EvaluatedType = Any def nullable = true From ad0fab23d810f0f62e71813edfc3101fcec5e40b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 27 Aug 2014 15:40:40 -0700 Subject: [PATCH 144/192] HOTFIX: Don't build with YARN support for Mapr3 --- dev/create-release/create-release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index eab6313733dfd..7549fbbe66654 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -121,7 +121,7 @@ make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Pyarn" & make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Pyarn" & make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & -make_binary_release "mapr3" "-Pmapr3 -Pyarn -Phive" & +make_binary_release "mapr3" "-Pmapr3 -Phive" & make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive" & wait From 79e86ef3e1a3ee03a7e3b166a5c7dee11c6d60d7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 06:45:54 +0000 Subject: [PATCH 145/192] [maven-release-plugin] prepare release v1.1.0-rc1 --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml From a118ea5c59d653f5a3feda21455ba60bc722b3b1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 06:46:02 +0000 Subject: [PATCH 146/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..7a9e680345bcc 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..83faf29de577f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..3661eac3e4f75 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..0fc7b580d9b9f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..692f87b1cb0ec 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..5d0f7ffc5390a 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..b267c4757623c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..c7fbf4b95500e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..36afc14c64fc5 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..5ba0fb1a4a7f3 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..a872bf2327fc6 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..430a5e7cbfde0 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..b3c843a5f3c4c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..f228212091b44 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..dd2e5116f5836 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..ef8d40d84285a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..909dc94230071 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..f6a5f955cdd64 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..b7540f02ce8ea 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..241e305cad986 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..09c403f08b16a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..09894f4eb07ea 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..bf298998e559f 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..c31eb0faf4368 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml From c0e3bc17ede69f792cd96556bb3d8acd98b239ce Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 28 Aug 2014 00:08:09 -0700 Subject: [PATCH 147/192] [SQL] Fixed 2 comment typos in SQLConf Author: Cheng Lian Closes #2172 from liancheng/sqlconf-typo and squashes the following commits: 115cc71 [Cheng Lian] Fixed 2 comment typos in SQLConf (cherry picked from commit 68f75dcdfe7e8ab229b73824692c4b3d4c39946c) Signed-off-by: Michael Armbrust --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index f0df19112ae37..64d49354dadcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -92,7 +92,7 @@ trait SQLConf { * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster * than interpreted evaluation, but there are significant start-up costs due to compilation. - * As a result codegen is only benificial when queries run for a long time, or when the same + * As a result codegen is only beneficial when queries run for a long time, or when the same * expressions are used multiple times. * * Defaults to false as this feature is currently experimental. @@ -111,8 +111,9 @@ trait SQLConf { /** * The default size in bytes to assign to a logical operator's estimation statistics. By default, - * it is set to a larger value than `autoConvertJoinSize`, hence any logical operator without a - * properly implemented estimation of this statistic will not be incorrectly broadcasted in joins. + * it is set to a larger value than `autoBroadcastJoinThreshold`, hence any logical operator + * without a properly implemented estimation of this statistic will not be incorrectly broadcasted + * in joins. */ private[spark] def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, (autoBroadcastJoinThreshold + 1).toString).toLong From 2e8ad9976c10a5a7fdefba9b612fc61e48f2b6cc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 28 Aug 2014 00:15:23 -0700 Subject: [PATCH 148/192] [SPARK-3230][SQL] Fix udfs that return structs We need to convert the case classes into Rows. Author: Michael Armbrust Closes #2133 from marmbrus/structUdfs and squashes the following commits: 189722f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into structUdfs 8e29b1c [Michael Armbrust] Use existing function d8d0b76 [Michael Armbrust] Fix udfs that return structs (cherry picked from commit 76e3ba4264c4a0bc2c33ae6ac862fc40bc302d83) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/catalyst/ScalaReflection.scala | 12 ++++++++++-- .../spark/sql/catalyst/expressions/ScalaUdf.scala | 7 ++++++- .../apache/spark/sql/execution/basicOperators.scala | 11 ++--------- .../test/scala/org/apache/spark/sql/UDFSuite.scala | 12 ++++++++++++ 4 files changed, 30 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 6b6b636cd96dc..88a8fa7c28e0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst import java.sql.Timestamp -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.types._ @@ -32,6 +31,15 @@ object ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) + /** Converts Scala objects to catalyst rows / types */ + def convertToCatalyst(a: Any): Any = a match { + case o: Option[_] => o.orNull + case s: Seq[_] => s.map(convertToCatalyst) + case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } + case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) + case other => other + } + /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 589816ccec0d5..1b687a443ef8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.util.ClosureCleaner @@ -27,6 +28,8 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def nullable = true + override def toString = s"scalaUDF(${children.mkString(",")})" + /** This method has been generated by this script (1 to 22).map { x => @@ -44,7 +47,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi // scalastyle:off override def eval(input: Row): Any = { - children.size match { + val result = children.size match { case 0 => function.asInstanceOf[() => Any]() case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => @@ -343,5 +346,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi children(21).eval(input)) } // scalastyle:on + + ScalaReflection.convertToCatalyst(result) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 374af48b820c2..4abda21ffec96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -204,14 +204,6 @@ case class Sort( */ @DeveloperApi object ExistingRdd { - def convertToCatalyst(a: Any): Any = a match { - case o: Option[_] => o.orNull - case s: Seq[_] => s.map(convertToCatalyst) - case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } - case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) - case other => other - } - def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { data.mapPartitions { iterator => if (iterator.isEmpty) { @@ -223,7 +215,7 @@ object ExistingRdd { bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = convertToCatalyst(r.productElement(i)) + mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) i += 1 } @@ -245,6 +237,7 @@ object ExistingRdd { case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } + /** * :: DeveloperApi :: * Computes the set of distinct input rows using a HashSet. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 76aa9b0081d7e..ef9b76b1e251e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,6 +22,8 @@ import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ +case class FunctionResult(f1: String, f2: String) + class UDFSuite extends QueryTest { test("Simple UDF") { @@ -33,4 +35,14 @@ class UDFSuite extends QueryTest { registerFunction("strLenScala", (_: String).length + (_:Int)) assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) } + + + test("struct UDF") { + registerFunction("returnStruct", (f1: String, f2: String) => FunctionResult(f1, f2)) + + val result= + sql("SELECT returnStruct('test', 'test2') as ret") + .select("ret.f1".attr).first().getString(0) + assert(result == "test") + } } From 71ec0140f7e121bdba3d19e8219e91a5e9d1e320 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 00:15:26 -0700 Subject: [PATCH 149/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit a118ea5c59d653f5a3feda21455ba60bc722b3b1. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7a9e680345bcc..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 83faf29de577f..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3661eac3e4f75..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 0fc7b580d9b9f..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 692f87b1cb0ec..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 5d0f7ffc5390a..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index b267c4757623c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index c7fbf4b95500e..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 36afc14c64fc5..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 5ba0fb1a4a7f3..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a872bf2327fc6..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 430a5e7cbfde0..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index b3c843a5f3c4c..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index f228212091b44..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index dd2e5116f5836..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index ef8d40d84285a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 909dc94230071..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f6a5f955cdd64..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index b7540f02ce8ea..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 241e305cad986..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 09c403f08b16a..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 09894f4eb07ea..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bf298998e559f..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index c31eb0faf4368..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml From 56070f12f455bae645cba887a74c72b12f1085f8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 00:15:30 -0700 Subject: [PATCH 150/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc1" This reverts commit 79e86ef3e1a3ee03a7e3b166a5c7dee11c6d60d7. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml From a9df703081449c41c32cc982781fe1922eaf91ad Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 00:19:03 -0700 Subject: [PATCH 151/192] Additional CHANGES.txt --- CHANGES.txt | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 6efb0225740a6..3a72a2089d271 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,36 @@ Spark Change Log Release 1.1.0 + [SPARK-3230][SQL] Fix udfs that return structs + Michael Armbrust + 2014-08-28 00:15:23 -0700 + Commit: 2e8ad99, github.com/apache/spark/pull/2133 + + [SQL] Fixed 2 comment typos in SQLConf + Cheng Lian + 2014-08-28 00:08:09 -0700 + Commit: c0e3bc1, github.com/apache/spark/pull/2172 + + HOTFIX: Don't build with YARN support for Mapr3 + Patrick Wendell + 2014-08-27 15:40:40 -0700 + Commit: ad0fab2 + + [HOTFIX][SQL] Remove cleaning of UDFs + Michael Armbrust + 2014-08-27 23:05:34 -0700 + Commit: 233c283, github.com/apache/spark/pull/2174 + + [HOTFIX] Wait for EOF only for the PySpark shell + Andrew Or + 2014-08-27 23:03:46 -0700 + Commit: 54ccd93, github.com/apache/spark/pull/2170 + + BUILD: Updating CHANGES.txt for Spark 1.1 + Patrick Wendell + 2014-08-27 15:55:59 -0700 + Commit: 8597e9c + Add line continuation for script to work w/ py2.7.5 Matthew Farrellee 2014-08-27 15:50:30 -0700 From da4b94c86c9dd0d624b3040aa4b9449be9f60fc3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 07:50:32 +0000 Subject: [PATCH 152/192] [maven-release-plugin] prepare release v1.1.0-rc1 --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml From 96926c5a42c5970ed74c50db5bd9c68cacf92207 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 07:50:43 +0000 Subject: [PATCH 153/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..7a9e680345bcc 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..83faf29de577f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..3661eac3e4f75 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..0fc7b580d9b9f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..692f87b1cb0ec 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..5d0f7ffc5390a 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..b267c4757623c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..c7fbf4b95500e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..36afc14c64fc5 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..5ba0fb1a4a7f3 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..a872bf2327fc6 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..430a5e7cbfde0 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..b3c843a5f3c4c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..f228212091b44 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..dd2e5116f5836 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..ef8d40d84285a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..909dc94230071 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..f6a5f955cdd64 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..b7540f02ce8ea 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..241e305cad986 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..09c403f08b16a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..09894f4eb07ea 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..bf298998e559f 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..c31eb0faf4368 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml From 473b02dfdd8e616717d9d10e4d88e3274e8c8e03 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:24 -0700 Subject: [PATCH 154/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 96926c5a42c5970ed74c50db5bd9c68cacf92207. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7a9e680345bcc..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 83faf29de577f..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3661eac3e4f75..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 0fc7b580d9b9f..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 692f87b1cb0ec..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 5d0f7ffc5390a..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index b267c4757623c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index c7fbf4b95500e..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 36afc14c64fc5..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 5ba0fb1a4a7f3..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a872bf2327fc6..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 430a5e7cbfde0..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index b3c843a5f3c4c..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index f228212091b44..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index dd2e5116f5836..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index ef8d40d84285a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 909dc94230071..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f6a5f955cdd64..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index b7540f02ce8ea..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 241e305cad986..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 09c403f08b16a..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 09894f4eb07ea..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bf298998e559f..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index c31eb0faf4368..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml From ecdbeeff8d62c6e0cdd4f998f76628e5692eb0c3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:31 -0700 Subject: [PATCH 155/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc1" This reverts commit da4b94c86c9dd0d624b3040aa4b9449be9f60fc3. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml From 4186c45f5bc3b72c87d45bdd791983d41968dc34 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:33 -0700 Subject: [PATCH 156/192] Revert "Revert "[maven-release-plugin] prepare release v1.1.0-rc1"" This reverts commit 56070f12f455bae645cba887a74c72b12f1085f8. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.1 ../pom.xml From df6194403a8431048521db1883f9b6feb9ec9983 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:36 -0700 Subject: [PATCH 157/192] Revert "Revert "[maven-release-plugin] prepare for next development iteration"" This reverts commit 71ec0140f7e121bdba3d19e8219e91a5e9d1e320. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..7a9e680345bcc 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..83faf29de577f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..3661eac3e4f75 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..0fc7b580d9b9f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..692f87b1cb0ec 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..5d0f7ffc5390a 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..b267c4757623c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..c7fbf4b95500e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..36afc14c64fc5 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..5ba0fb1a4a7f3 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..a872bf2327fc6 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..430a5e7cbfde0 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..b3c843a5f3c4c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..f228212091b44 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..dd2e5116f5836 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..ef8d40d84285a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..909dc94230071 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..f6a5f955cdd64 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..b7540f02ce8ea 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..241e305cad986 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..09c403f08b16a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..09894f4eb07ea 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..bf298998e559f 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..c31eb0faf4368 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.2-SNAPSHOT ../pom.xml From d01b3faf896c7e5e26bd1a8d1f9f125097e4dd6f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:41 -0700 Subject: [PATCH 158/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit a118ea5c59d653f5a3feda21455ba60bc722b3b1. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7a9e680345bcc..5896b6f7795e3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 83faf29de577f..f1c2b2171e010 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3661eac3e4f75..4e094bde24b5b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 0fc7b580d9b9f..eb7fd0ab04cad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 692f87b1cb0ec..c443eaa238407 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 5d0f7ffc5390a..6d06a2da640f3 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index b267c4757623c..09602f672516b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index c7fbf4b95500e..462079cbf2eb8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 36afc14c64fc5..c1628831bf258 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 5ba0fb1a4a7f3..7670646bc6ea5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a872bf2327fc6..2433818b07be0 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 430a5e7cbfde0..b6f4875f9d7ca 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index b3c843a5f3c4c..cb3d09038e4da 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index f228212091b44..9443f6118b3bb 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/pom.xml b/pom.xml index dd2e5116f5836..27b0bbbab69f7 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index ef8d40d84285a..b0b9196bc7b86 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 909dc94230071..19ca3044f987e 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f6a5f955cdd64..148a521d6e5e0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index b7540f02ce8ea..c3c2f552f25e1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 241e305cad986..ff01bc6dc7f8c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 09c403f08b16a..9b56e32764a5f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 09894f4eb07ea..a2bb5af01dcd3 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bf298998e559f..2fad21bb08505 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index c31eb0faf4368..74a30ae8a7af9 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.2-SNAPSHOT + 1.1.1 ../pom.xml From c818b2b0408c42bf7a9ce933e83bcd5e35c42817 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:44 -0700 Subject: [PATCH 159/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc1" This reverts commit 79e86ef3e1a3ee03a7e3b166a5c7dee11c6d60d7. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5896b6f7795e3..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f1c2b2171e010..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 4e094bde24b5b..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index eb7fd0ab04cad..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c443eaa238407..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6d06a2da640f3..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 09602f672516b..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 462079cbf2eb8..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index c1628831bf258..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7670646bc6ea5..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 2433818b07be0..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index b6f4875f9d7ca..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index cb3d09038e4da..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9443f6118b3bb..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 27b0bbbab69f7..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b0b9196bc7b86..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 19ca3044f987e..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 148a521d6e5e0..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c3c2f552f25e1..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ff01bc6dc7f8c..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b56e32764a5f..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2bb5af01dcd3..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 2fad21bb08505..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 74a30ae8a7af9..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1 + 1.1.1-SNAPSHOT ../pom.xml From c0bacc161a2c6e252a62ff3369e6644bb9ef9da6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:46 -0700 Subject: [PATCH 160/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 78e3c036eee7113b2ed144eec5061e070b479e56. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..b861e77c387f6 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml From 1d03330ec40f3818f4fb226f79c17111f17a1b34 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 01:55:48 -0700 Subject: [PATCH 161/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc1" This reverts commit 58b0be6a29eab817d350729710345e9f39e4c506. --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 +++++---- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 +++-- sql/core/pom.xml | 5 +++-- sql/hive-thriftserver/pom.xml | 5 +++-- sql/hive/pom.xml | 5 +++-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 38 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..a54b34235dfb4 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index b861e77c387f6..c5a9b3d5f4fd0 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD @@ -880,7 +881,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From f07183249b74dd857069028bf7d570b35f265585 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 09:29:24 +0000 Subject: [PATCH 162/192] [maven-release-plugin] prepare release v1.1.0-rc1 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index c5a9b3d5f4fd0..b861e77c387f6 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 @@ -881,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From f8f7a0c9dce764ece8acdc41d35bbf448dba7e92 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 09:29:30 +0000 Subject: [PATCH 163/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index b861e77c387f6..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From fd98020a9a9880a09147f01d8480cab204e1c5d7 Mon Sep 17 00:00:00 2001 From: Tatiana Borisova Date: Thu, 28 Aug 2014 10:36:36 -0700 Subject: [PATCH 164/192] [SPARK-3150] Fix NullPointerException in in Spark recovery: Add initializing default values in DriverInfo.init() The issue happens when Spark is run standalone on a cluster. When master and driver fall simultaneously on one node in a cluster, master tries to recover its state and restart spark driver. While restarting driver, it falls with NPE exception (stacktrace is below). After falling, it restarts and tries to recover its state and restart Spark driver again. It happens over and over in an infinite cycle. Namely, Spark tries to read DriverInfo state from zookeeper, but after reading it happens to be null in DriverInfo.worker. https://issues.apache.org/jira/browse/SPARK-3150 Author: Tatiana Borisova Closes #2062 from tanyatik/spark-3150 and squashes the following commits: 9936043 [Tatiana Borisova] Add initializing default values in DriverInfo.init() (cherry picked from commit 70d814665baa8b8ca868d3126452105ecfa5cbff) Signed-off-by: Josh Rosen --- .../org/apache/spark/deploy/master/DriverInfo.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 33377931d6993..80b570a44af18 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -33,4 +33,17 @@ private[spark] class DriverInfo( @transient var exception: Option[Exception] = None /* Most recent worker assigned to this driver */ @transient var worker: Option[WorkerInfo] = None + + init() + + private def readObject(in: java.io.ObjectInputStream): Unit = { + in.defaultReadObject() + init() + } + + private def init(): Unit = { + state = DriverState.SUBMITTED + worker = None + exception = None + } } From 069ecfef02c4af69fc0d3755bd78be321b68b01d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 28 Aug 2014 11:05:44 -0700 Subject: [PATCH 165/192] [SPARK-3264] Allow users to set executor Spark home in Mesos The executors and the driver may not share the same Spark home. There is currently one way to set the executor side Spark home in Mesos, through setting `spark.home`. However, this is neither documented nor intuitive. This PR adds a more specific config `spark.mesos.executor.home` and exposes this to the user. liancheng tnachen Author: Andrew Or Closes #2166 from andrewor14/mesos-spark-home and squashes the following commits: b87965e [Andrew Or] Merge branch 'master' of github.com:apache/spark into mesos-spark-home f6abb2e [Andrew Or] Document spark.mesos.executor.home ca7846d [Andrew Or] Add more specific configuration for executor Spark home in Mesos (cherry picked from commit 41dc5987d9abeca6fc0f5935c780d48f517cdf95) Signed-off-by: Andrew Or --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 10 ++++++---- .../cluster/mesos/MesosSchedulerBackend.scala | 10 ++++++---- docs/configuration.md | 10 ++++++++++ 3 files changed, 22 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 8c7cb07ebc5ba..87e181e773fdf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -71,9 +71,11 @@ private[spark] class CoarseMesosSchedulerBackend( val taskIdToSlaveId = new HashMap[Int, String] val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed - val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( - "Spark home is not set; set it through the spark.home system " + - "property, the SPARK_HOME environment variable or the SparkContext constructor")) + val executorSparkHome = conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) @@ -144,7 +146,7 @@ private[spark] class CoarseMesosSchedulerBackend( val uri = conf.get("spark.executor.uri", null) if (uri == null) { - val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath + val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index e84ce094e5eec..67ee4d66f151b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -86,9 +86,11 @@ private[spark] class MesosSchedulerBackend( } def createExecutorInfo(execId: String): ExecutorInfo = { - val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( - "Spark home is not set; set it through the spark.home system " + - "property, the SPARK_HOME environment variable or the SparkContext constructor")) + val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val environment = Environment.newBuilder() sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => environment.addVariables( @@ -114,7 +116,7 @@ private[spark] class MesosSchedulerBackend( .setEnvironment(environment) val uri = sc.conf.get("spark.executor.uri", null) if (uri == null) { - command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath) + command.setValue(new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". diff --git a/docs/configuration.md b/docs/configuration.md index 981170d8b49b7..65a422caabb7e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -214,6 +214,16 @@ Apart from these, the following properties are also available, and may be useful process. The user can specify multiple of these and to set multiple environment variables. + + spark.mesos.executor.home + driver side SPARK_HOME + + Set the directory in which Spark is installed on the executors in Mesos. By default, the + executors will simply use the driver's Spark home directory, which may not be visible to + them. Note that this is only relevant if a Spark binary package is not specified through + spark.executor.uri. + + #### Shuffle Behavior From 0b9718add107aa7cc9528fea177ae8239158014b Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 28 Aug 2014 15:17:01 -0700 Subject: [PATCH 166/192] [SPARK-3190] Avoid overflow in VertexRDD.count() VertexRDDs with more than 4 billion elements are counted incorrectly due to integer overflow when summing partition sizes. This PR fixes the issue by converting partition sizes to Longs before summing them. The following code previously returned -10000000. After applying this PR, it returns the correct answer of 5000000000 (5 billion). ```scala val pairs = sc.parallelize(0L until 500L).map(_ * 10000000) .flatMap(start => start until (start + 10000000)).map(x => (x, x)) VertexRDD(pairs).count() ``` Author: Ankur Dave Closes #2106 from ankurdave/SPARK-3190 and squashes the following commits: 641f468 [Ankur Dave] Avoid overflow in VertexRDD.count() (cherry picked from commit 96df92906978c5f58e0cc8ff5eebe5b35a08be3b) Signed-off-by: Josh Rosen --- graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 4825d12fc27b3..04fbc9dbab8d1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -108,7 +108,7 @@ class VertexRDD[@specialized VD: ClassTag]( /** The number of vertices in the RDD. */ override def count(): Long = { - partitionsRDD.map(_.size).reduce(_ + _) + partitionsRDD.map(_.size.toLong).reduce(_ + _) } /** From f4cbf5e3d55d8ae66bfb1dde80cef81a2dfad7e4 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 28 Aug 2014 16:18:50 -0700 Subject: [PATCH 167/192] SPARK-3082. yarn.Client.logClusterResourceDetails throws NPE if requeste... ...d queue doesn't exist Author: Sandy Ryza Closes #1984 from sryza/sandy-spark-3082 and squashes the following commits: fe08c37 [Sandy Ryza] Remove log message entirely 85253ad [Sandy Ryza] SPARK-3082. yarn.Client.logClusterResourceDetails throws NPE if requested queue doesn't exist (cherry picked from commit 92af2314f27e80227174499f2fca505bd551cda7) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/deploy/yarn/Client.scala | 11 +---------- .../scala/org/apache/spark/deploy/yarn/Client.scala | 11 +---------- 2 files changed, 2 insertions(+), 20 deletions(-) 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 82f79d88a3009..9be78546c1091 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 @@ -89,17 +89,8 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def logClusterResourceDetails() { val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got Cluster metric info from ASM, numNodeManagers = " + + logInfo("Got cluster metric info from ASM, numNodeManagers = " + clusterMetrics.getNumNodeManagers) - - val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo( """Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, - queueApplicationCount = %s, queueChildQueueCount = %s""".format( - queueInfo.getQueueName, - queueInfo.getCurrentCapacity, - queueInfo.getMaximumCapacity, - queueInfo.getApplications.size, - queueInfo.getChildQueues.size)) } 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 15f3c4f180ea3..1f9a4bf209eb9 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 @@ -99,17 +99,8 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def logClusterResourceDetails() { val clusterMetrics: YarnClusterMetrics = yarnClient.getYarnClusterMetrics - logInfo("Got Cluster metric info from ResourceManager, number of NodeManagers: " + + logInfo("Got cluster metric info from ResourceManager, number of NodeManagers: " + clusterMetrics.getNumNodeManagers) - - val queueInfo: QueueInfo = yarnClient.getQueueInfo(args.amQueue) - logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, - queueApplicationCount = %s, queueChildQueueCount = %s""".format( - queueInfo.getQueueName, - queueInfo.getCurrentCapacity, - queueInfo.getMaximumCapacity, - queueInfo.getApplications.size, - queueInfo.getChildQueues.size)) } def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = { From fe4df343d40e20eb575e9ffabd8f4a9ae3952f24 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 28 Aug 2014 17:05:21 -0700 Subject: [PATCH 168/192] [SPARK-3277] Fix external spilling with LZ4 assertion error **Summary of the changes** The bulk of this PR is comprised of tests and documentation; the actual fix is really just adding 1 line of code (see `BlockObjectWriter.scala`). We currently do not run the `External*` test suites with different compression codecs, and this would have caught the bug reported in [SPARK-3277](https://issues.apache.org/jira/browse/SPARK-3277). This PR extends the existing code to test spilling using all compression codecs known to Spark, including `LZ4`. **The bug itself** In `DiskBlockObjectWriter`, we only report the shuffle bytes written before we close the streams. With `LZ4`, all the bytes written reported by our metrics were 0 because `flush()` was not taking effect for some reason. In general, compression codecs may write additional bytes to the file after we call `close()`, and so we must also capture those bytes in our shuffle write metrics. Thanks mridulm and pwendell for help with debugging. Author: Andrew Or Author: Patrick Wendell Closes #2187 from andrewor14/fix-lz4-spilling and squashes the following commits: 1b54bdc [Andrew Or] Speed up tests by not compressing everything 1c4624e [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-lz4-spilling 6b2e7d1 [Andrew Or] Fix compilation error 92e251b [Patrick Wendell] Better documentation for BlockObjectWriter. a1ad536 [Andrew Or] Fix tests 089593f [Andrew Or] Actually fix SPARK-3277 (tests still fail) 4bbcf68 [Andrew Or] Update tests to actually test all compression codecs b264a84 [Andrew Or] ExternalAppendOnlyMapSuite code style fixes (minor) 1bfa743 [Andrew Or] Add more information to assert for better debugging --- .../apache/spark/io/CompressionCodec.scala | 1 + .../spark/storage/BlockObjectWriter.scala | 37 +++- .../collection/ExternalAppendOnlyMap.scala | 7 +- .../spark/scheduler/ReplayListenerSuite.scala | 5 +- .../ExternalAppendOnlyMapSuite.scala | 190 ++++++++++-------- 5 files changed, 144 insertions(+), 96 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index ef9c43ecf14f6..1ac7f4e448eb1 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -64,6 +64,7 @@ private[spark] object CompressionCodec { } val DEFAULT_COMPRESSION_CODEC = "snappy" + val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index adda971fd7b47..9c469370ffe1f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -65,8 +65,6 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { /** * BlockObjectWriter which writes directly to a file on disk. Appends to the given file. - * The given write metrics will be updated incrementally, but will not necessarily be current until - * commitAndClose is called. */ private[spark] class DiskBlockObjectWriter( blockId: BlockId, @@ -75,6 +73,8 @@ private[spark] class DiskBlockObjectWriter( bufferSize: Int, compressStream: OutputStream => OutputStream, syncWrites: Boolean, + // These write metrics concurrently shared with other active BlockObjectWriter's who + // are themselves performing writes. All updates must be relative. writeMetrics: ShuffleWriteMetrics) extends BlockObjectWriter(blockId) with Logging @@ -94,14 +94,30 @@ private[spark] class DiskBlockObjectWriter( private var fos: FileOutputStream = null private var ts: TimeTrackingOutputStream = null private var objOut: SerializationStream = null + private var initialized = false + + /** + * Cursors used to represent positions in the file. + * + * xxxxxxxx|--------|--- | + * ^ ^ ^ + * | | finalPosition + * | reportedPosition + * initialPosition + * + * initialPosition: Offset in the file where we start writing. Immutable. + * reportedPosition: Position at the time of the last update to the write metrics. + * finalPosition: Offset where we stopped writing. Set on closeAndCommit() then never changed. + * -----: Current writes to the underlying file. + * xxxxx: Existing contents of the file. + */ private val initialPosition = file.length() private var finalPosition: Long = -1 - private var initialized = false + private var reportedPosition = initialPosition /** Calling channel.position() to update the write metrics can be a little bit expensive, so we * only call it every N writes */ private var writesSinceMetricsUpdate = 0 - private var lastPosition = initialPosition override def open(): BlockObjectWriter = { fos = new FileOutputStream(file, true) @@ -140,17 +156,18 @@ private[spark] class DiskBlockObjectWriter( // serializer stream and the lower level stream. objOut.flush() bs.flush() - updateBytesWritten() close() } finalPosition = file.length() + // In certain compression codecs, more bytes are written after close() is called + writeMetrics.shuffleBytesWritten += (finalPosition - reportedPosition) } // Discard current writes. We do this by flushing the outstanding writes and then // truncating the file to its initial position. override def revertPartialWritesAndClose() { try { - writeMetrics.shuffleBytesWritten -= (lastPosition - initialPosition) + writeMetrics.shuffleBytesWritten -= (reportedPosition - initialPosition) if (initialized) { objOut.flush() @@ -189,10 +206,14 @@ private[spark] class DiskBlockObjectWriter( new FileSegment(file, initialPosition, finalPosition - initialPosition) } + /** + * Report the number of bytes written in this writer's shuffle write metrics. + * Note that this is only valid before the underlying streams are closed. + */ private def updateBytesWritten() { val pos = channel.position() - writeMetrics.shuffleBytesWritten += (pos - lastPosition) - lastPosition = pos + writeMetrics.shuffleBytesWritten += (pos - reportedPosition) + reportedPosition = pos } private def callWithTiming(f: => Unit) = { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 9f85b94a70800..8a015c1d26a96 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -413,7 +413,12 @@ class ExternalAppendOnlyMap[K, V, C]( extends Iterator[(K, C)] { private val batchOffsets = batchSizes.scanLeft(0L)(_ + _) // Size will be batchSize.length + 1 - assert(file.length() == batchOffsets(batchOffsets.length - 1)) + assert(file.length() == batchOffsets.last, + "File length is not equal to the last batch offset:\n" + + s" file length = ${file.length}\n" + + s" last batch offset = ${batchOffsets.last}\n" + + s" all batch offsets = ${batchOffsets.mkString(",")}" + ) private var batchIndex = 0 // Which batch we're in private var fileStream: FileInputStream = null diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d81499ac6abef..6b6e0104e5467 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -33,10 +33,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { private val fileSystem = Utils.getHadoopFileSystem("/") - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) + private val allCompressionCodecs = CompressionCodec.ALL_COMPRESSION_CODECS private var testDir: File = _ before { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 04d7338488628..ac3931e3d0a73 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -23,37 +23,42 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.SparkContext._ +import org.apache.spark.io.CompressionCodec class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { + private val allCompressionCodecs = CompressionCodec.ALL_COMPRESSION_CODECS + private def createCombiner[T](i: T) = ArrayBuffer[T](i) + private def mergeValue[T](buffer: ArrayBuffer[T], i: T): ArrayBuffer[T] = buffer += i + private def mergeCombiners[T](buf1: ArrayBuffer[T], buf2: ArrayBuffer[T]): ArrayBuffer[T] = + buf1 ++= buf2 - private def createCombiner(i: Int) = ArrayBuffer[Int](i) - private def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i - private def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + private def createExternalMap[T] = new ExternalAppendOnlyMap[T, T, ArrayBuffer[T]]( + createCombiner[T], mergeValue[T], mergeCombiners[T]) - private def createSparkConf(loadDefaults: Boolean): SparkConf = { + private def createSparkConf(loadDefaults: Boolean, codec: Option[String] = None): SparkConf = { val conf = new SparkConf(loadDefaults) // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) conf.set("spark.serializer.objectStreamReset", "1") conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + conf.set("spark.shuffle.spill.compress", codec.isDefined.toString) + codec.foreach { c => conf.set("spark.io.compression.codec", c) } // Ensure that we actually have multiple batches per spill file conf.set("spark.shuffle.spill.batchSize", "10") conf } test("simple insert") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) - - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map = createExternalMap[Int] // Single insert map.insert(1, 10) var it = map.iterator assert(it.hasNext) val kv = it.next() - assert(kv._1 == 1 && kv._2 == ArrayBuffer[Int](10)) + assert(kv._1 === 1 && kv._2 === ArrayBuffer[Int](10)) assert(!it.hasNext) // Multiple insert @@ -61,18 +66,17 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { map.insert(3, 30) it = map.iterator assert(it.hasNext) - assert(it.toSet == Set[(Int, ArrayBuffer[Int])]( + assert(it.toSet === Set[(Int, ArrayBuffer[Int])]( (1, ArrayBuffer[Int](10)), (2, ArrayBuffer[Int](20)), (3, ArrayBuffer[Int](30)))) + sc.stop() } test("insert with collision") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) - - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map = createExternalMap[Int] map.insertAll(Seq( (1, 10), @@ -84,30 +88,28 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) - assert(result == Set[(Int, Set[Int])]( + assert(result === Set[(Int, Set[Int])]( (1, Set[Int](10, 100, 1000)), (2, Set[Int](20, 200)), (3, Set[Int](30)))) + sc.stop() } test("ordering") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) - val map1 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map1 = createExternalMap[Int] map1.insert(1, 10) map1.insert(2, 20) map1.insert(3, 30) - val map2 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map2 = createExternalMap[Int] map2.insert(2, 20) map2.insert(3, 30) map2.insert(1, 10) - val map3 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map3 = createExternalMap[Int] map3.insert(3, 30) map3.insert(1, 10) map3.insert(2, 20) @@ -119,33 +121,33 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { var kv1 = it1.next() var kv2 = it2.next() var kv3 = it3.next() - assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) - assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + assert(kv1._1 === kv2._1 && kv2._1 === kv3._1) + assert(kv1._2 === kv2._2 && kv2._2 === kv3._2) kv1 = it1.next() kv2 = it2.next() kv3 = it3.next() - assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) - assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + assert(kv1._1 === kv2._1 && kv2._1 === kv3._1) + assert(kv1._2 === kv2._2 && kv2._2 === kv3._2) kv1 = it1.next() kv2 = it2.next() kv3 = it3.next() - assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) - assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + assert(kv1._1 === kv2._1 && kv2._1 === kv3._1) + assert(kv1._2 === kv2._2 && kv2._2 === kv3._2) + sc.stop() } test("null keys and values") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map = createExternalMap[Int] map.insert(1, 5) map.insert(2, 6) map.insert(3, 7) assert(map.size === 3) - assert(map.iterator.toSet == Set[(Int, Seq[Int])]( + assert(map.iterator.toSet === Set[(Int, Seq[Int])]( (1, Seq[Int](5)), (2, Seq[Int](6)), (3, Seq[Int](7)) @@ -155,7 +157,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val nullInt = null.asInstanceOf[Int] map.insert(nullInt, 8) assert(map.size === 4) - assert(map.iterator.toSet == Set[(Int, Seq[Int])]( + assert(map.iterator.toSet === Set[(Int, Seq[Int])]( (1, Seq[Int](5)), (2, Seq[Int](6)), (3, Seq[Int](7)), @@ -167,32 +169,34 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { map.insert(nullInt, nullInt) assert(map.size === 5) val result = map.iterator.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) - assert(result == Set[(Int, Set[Int])]( + assert(result === Set[(Int, Set[Int])]( (1, Set[Int](5)), (2, Set[Int](6)), (3, Set[Int](7)), (4, Set[Int](nullInt)), (nullInt, Set[Int](nullInt, 8)) )) + sc.stop() } test("simple aggregator") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) // reduceByKey val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) val result1 = rdd.reduceByKey(_+_).collect() - assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) + assert(result1.toSet === Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet - assert(result2.toSet == Set[(Int, Seq[Int])] + assert(result2.toSet === Set[(Int, Seq[Int])] ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) + sc.stop() } test("simple cogroup") { - val conf = createSparkConf(false) + val conf = createSparkConf(loadDefaults = false) sc = new SparkContext("local", "test", conf) val rdd1 = sc.parallelize(1 to 4).map(i => (i, i)) val rdd2 = sc.parallelize(1 to 4).map(i => (i%2, i)) @@ -200,77 +204,98 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { result.foreach { case (i, (seq1, seq2)) => i match { - case 0 => assert(seq1.toSet == Set[Int]() && seq2.toSet == Set[Int](2, 4)) - case 1 => assert(seq1.toSet == Set[Int](1) && seq2.toSet == Set[Int](1, 3)) - case 2 => assert(seq1.toSet == Set[Int](2) && seq2.toSet == Set[Int]()) - case 3 => assert(seq1.toSet == Set[Int](3) && seq2.toSet == Set[Int]()) - case 4 => assert(seq1.toSet == Set[Int](4) && seq2.toSet == Set[Int]()) + case 0 => assert(seq1.toSet === Set[Int]() && seq2.toSet === Set[Int](2, 4)) + case 1 => assert(seq1.toSet === Set[Int](1) && seq2.toSet === Set[Int](1, 3)) + case 2 => assert(seq1.toSet === Set[Int](2) && seq2.toSet === Set[Int]()) + case 3 => assert(seq1.toSet === Set[Int](3) && seq2.toSet === Set[Int]()) + case 4 => assert(seq1.toSet === Set[Int](4) && seq2.toSet === Set[Int]()) } } + sc.stop() } test("spilling") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + testSimpleSpilling() + } + + test("spilling with compression") { + // Keep track of which compression codec we're using to report in test failure messages + var lastCompressionCodec: Option[String] = None + try { + allCompressionCodecs.foreach { c => + lastCompressionCodec = Some(c) + testSimpleSpilling(Some(c)) + } + } catch { + // Include compression codec used in test failure message + // We need to catch Throwable here because assertion failures are not covered by Exceptions + case t: Throwable => + val compressionMessage = lastCompressionCodec + .map { c => "with compression using codec " + c } + .getOrElse("without compression") + val newException = new Exception(s"Test failed $compressionMessage:\n\n${t.getMessage}") + newException.setStackTrace(t.getStackTrace) + throw newException + } + } + + /** + * Test spilling through simple aggregations and cogroups. + * If a compression codec is provided, use it. Otherwise, do not compress spills. + */ + private def testSimpleSpilling(codec: Option[String] = None): Unit = { + val conf = createSparkConf(loadDefaults = true, codec) // Load defaults for Spark home conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) // reduceByKey - should spill ~8 times val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) val resultA = rddA.reduceByKey(math.max).collect() - assert(resultA.length == 50000) - resultA.foreach { case(k, v) => - if (v != k * 2 + 1) { - fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") - } + assert(resultA.length === 50000) + resultA.foreach { case (k, v) => + assert(v === k * 2 + 1, s"Value for $k was wrong: expected ${k * 2 + 1}, got $v") } // groupByKey - should spill ~17 times val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i)) val resultB = rddB.groupByKey().collect() - assert(resultB.length == 25000) - resultB.foreach { case(i, seq) => + assert(resultB.length === 25000) + resultB.foreach { case (i, seq) => val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) - if (seq.toSet != expected) { - fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") - } + assert(seq.toSet === expected, + s"Value for $i was wrong: expected $expected, got ${seq.toSet}") } // cogroup - should spill ~7 times val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i)) val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i)) val resultC = rddC1.cogroup(rddC2).collect() - assert(resultC.length == 10000) - resultC.foreach { case(i, (seq1, seq2)) => + assert(resultC.length === 10000) + resultC.foreach { case (i, (seq1, seq2)) => i match { case 0 => - assert(seq1.toSet == Set[Int](0)) - assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + assert(seq1.toSet === Set[Int](0)) + assert(seq2.toSet === Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) case 1 => - assert(seq1.toSet == Set[Int](1)) - assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) + assert(seq1.toSet === Set[Int](1)) + assert(seq2.toSet === Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) case 5000 => - assert(seq1.toSet == Set[Int](5000)) - assert(seq2.toSet == Set[Int]()) + assert(seq1.toSet === Set[Int](5000)) + assert(seq2.toSet === Set[Int]()) case 9999 => - assert(seq1.toSet == Set[Int](9999)) - assert(seq2.toSet == Set[Int]()) + assert(seq1.toSet === Set[Int](9999)) + assert(seq2.toSet === Set[Int]()) case _ => } } + sc.stop() } test("spilling with hash collisions") { - val conf = createSparkConf(true) + val conf = createSparkConf(loadDefaults = true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = - buffer1 ++= buffer2 - - val map = new ExternalAppendOnlyMap[String, String, ArrayBuffer[String]]( - createCombiner, mergeValue, mergeCombiners) + val map = createExternalMap[String] val collisionPairs = Seq( ("Aa", "BB"), // 2112 @@ -312,13 +337,13 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { count += 1 } assert(count === 100000 + collisionPairs.size * 2) + sc.stop() } test("spilling with many hash collisions") { - val conf = createSparkConf(true) + val conf = createSparkConf(loadDefaults = true) conf.set("spark.shuffle.memoryFraction", "0.0001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - val map = new ExternalAppendOnlyMap[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes @@ -337,15 +362,14 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { count += 1 } assert(count === 10000) + sc.stop() } test("spilling with hash collisions using the Int.MaxValue key") { - val conf = createSparkConf(true) + val conf = createSparkConf(loadDefaults = true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( - createCombiner, mergeValue, mergeCombiners) + val map = createExternalMap[Int] (1 to 100000).foreach { i => map.insert(i, i) } map.insert(Int.MaxValue, Int.MaxValue) @@ -355,15 +379,14 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { // Should not throw NoSuchElementException it.next() } + sc.stop() } test("spilling with null keys and values") { - val conf = createSparkConf(true) + val conf = createSparkConf(loadDefaults = true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( - createCombiner, mergeValue, mergeCombiners) + val map = createExternalMap[Int] map.insertAll((1 to 100000).iterator.map(i => (i, i))) map.insert(null.asInstanceOf[Int], 1) @@ -375,6 +398,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { // Should not throw NullPointerException it.next() } + sc.stop() } } From 7db87b3ec7aa7cc70031a95fa1fd1143ba0efd71 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 17:17:30 -0700 Subject: [PATCH 169/192] Adding new CHANGES.txt --- CHANGES.txt | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 3a72a2089d271..b42721ac45bfe 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,36 @@ Spark Change Log Release 1.1.0 + [SPARK-3277] Fix external spilling with LZ4 assertion error + Andrew Or , Patrick Wendell + 2014-08-28 17:05:21 -0700 + Commit: fe4df34, github.com/apache/spark/pull/2187 + + SPARK-3082. yarn.Client.logClusterResourceDetails throws NPE if requeste... + Sandy Ryza + 2014-08-28 16:18:50 -0700 + Commit: f4cbf5e, github.com/apache/spark/pull/1984 + + [SPARK-3190] Avoid overflow in VertexRDD.count() + Ankur Dave + 2014-08-28 15:17:01 -0700 + Commit: 0b9718a, github.com/apache/spark/pull/2106 + + [SPARK-3264] Allow users to set executor Spark home in Mesos + Andrew Or + 2014-08-28 11:05:44 -0700 + Commit: 069ecfe, github.com/apache/spark/pull/2166 + + [SPARK-3150] Fix NullPointerException in in Spark recovery: Add initializing default values in DriverInfo.init() + Tatiana Borisova + 2014-08-28 10:36:36 -0700 + Commit: fd98020, github.com/apache/spark/pull/2062 + + Additional CHANGES.txt + Patrick Wendell + 2014-08-28 00:19:03 -0700 + Commit: a9df703 + [SPARK-3230][SQL] Fix udfs that return structs Michael Armbrust 2014-08-28 00:15:23 -0700 From 587dff2133945e07bb0e3f0a3b4746f337187bbe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 17:18:20 -0700 Subject: [PATCH 170/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit f8f7a0c9dce764ece8acdc41d35bbf448dba7e92. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..b861e77c387f6 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc1 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml From fb2b40a52f3cf1a6c638d84951119aeb2c61b29d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 28 Aug 2014 17:18:28 -0700 Subject: [PATCH 171/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc1" This reverts commit f07183249b74dd857069028bf7d570b35f265585. --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 +++++---- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 +++-- sql/core/pom.xml | 5 +++-- sql/hive-thriftserver/pom.xml | 5 +++-- sql/hive/pom.xml | 5 +++-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 38 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..a54b34235dfb4 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index b861e77c387f6..c5a9b3d5f4fd0 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc1 + HEAD @@ -880,7 +881,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From 711aebb329ca28046396af1e34395a0df92b5327 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 00:54:02 +0000 Subject: [PATCH 172/192] [maven-release-plugin] prepare release v1.1.0-rc2 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index c5a9b3d5f4fd0..c7dc7efc6eb7d 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc2 @@ -881,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From a4a7a241441489a0d31365e18476ae2e1c34464d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 00:54:09 +0000 Subject: [PATCH 173/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index c7dc7efc6eb7d..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc2 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From c71b5c6db151cfc63bfeabdc88034c3dd9dc9e60 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 11:20:45 -0700 Subject: [PATCH 174/192] HOTFIX: Bump spark-ec2 version to 1.1.0 --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index afef4ef9645ca..59356c5dd96ec 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -82,7 +82,7 @@ def parse_args(): "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") parser.add_option( - "-v", "--spark-version", default="1.0.0", + "-v", "--spark-version", default="1.1.0", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option( "--spark-git-repo", From 98d0716a11e8e5731adfa6031a5d35916f42b6f5 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 29 Aug 2014 11:47:49 -0700 Subject: [PATCH 175/192] [SPARK-3307] [PySpark] Fix doc string of SparkContext.broadcast() remove invalid docs Author: Davies Liu Closes #2202 from davies/keep and squashes the following commits: aa3b44f [Davies Liu] remove invalid docs (cherry picked from commit e248328b39f52073422a12fd0388208de41be1c7) Signed-off-by: Josh Rosen --- python/pyspark/context.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a90870ed3a353..82f76de31afc1 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -568,8 +568,6 @@ def broadcast(self, value): L{Broadcast} object for reading it in distributed functions. The variable will be sent to each cluster only once. - - :keep: Keep the `value` in driver or not. """ ser = CompressedSerializer(PickleSerializer()) # pass large object by py4j is very slow and need much memory From bfa2dc99a22c23dc4b10d1f9e5dd9681f6f48537 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 29 Aug 2014 15:23:32 -0700 Subject: [PATCH 176/192] [Docs] SQL doc formatting and typo fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit As [reported on the dev list](http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-Release-Apache-Spark-1-1-0-RC2-tp8107p8131.html): * Code fencing with triple-backticks doesn’t seem to work like it does on GitHub. Newlines are lost. Instead, use 4-space indent to format small code blocks. * Nested bullets need 2 leading spaces, not 1. * Spellcheck! Author: Nicholas Chammas Author: nchammas Closes #2201 from nchammas/sql-doc-fixes and squashes the following commits: 873f889 [Nicholas Chammas] [Docs] fix skip-api flag 5195e0c [Nicholas Chammas] [Docs] SQL doc formatting and typo fixes 3b26c8d [nchammas] [Spark QA] Link to console output on test time out (cherry picked from commit 53aa8316e88980c6f46d3b9fc90d935a4738a370) Signed-off-by: Michael Armbrust --- docs/README.md | 2 +- docs/sql-programming-guide.md | 109 ++++++++++++++++------------------ 2 files changed, 52 insertions(+), 59 deletions(-) diff --git a/docs/README.md b/docs/README.md index fd7ba4e0d72ea..0a0126c5747d1 100644 --- a/docs/README.md +++ b/docs/README.md @@ -30,7 +30,7 @@ called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: # Skip generating API docs (which takes a while) - $ SKIP_SCALADOC=1 jekyll build + $ SKIP_API=1 jekyll build # Serve content locally on port 4000 $ jekyll serve --watch # Build the site with extra features used on the live page diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c41f2804a6021..8f7fb5431cfb6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -474,10 +474,10 @@ anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`sbt/sbt -Phive assembly/assembly`' (or use `-Phive` for maven). +In order to use Hive you must first run "`sbt/sbt -Phive assembly/assembly`" (or use `-Phive` for maven). This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries -(SerDes) in order to acccess data stored in Hive. +(SerDes) in order to access data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. @@ -576,9 +576,8 @@ evaluated by the SQL execution engine. A full list of the functions supported c ## Running the Thrift JDBC server -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] -(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. +The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +in Hive 0.12. You can test the JDBC server with the beeline script comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: @@ -597,7 +596,7 @@ Connect to the JDBC server in beeline with: Beeline will ask you for a username and password. In non-secure mode, simply enter the username on your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. @@ -616,11 +615,10 @@ In Shark, default reducer number is 1 and is controlled by the property `mapred. SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value is 200. Users may customize this property via `SET`: -``` -SET spark.sql.shuffle.partitions=10; -SELECT page, count(*) c FROM logs_last_month_cached -GROUP BY page ORDER BY c DESC LIMIT 10; -``` + SET spark.sql.shuffle.partitions=10; + SELECT page, count(*) c + FROM logs_last_month_cached + GROUP BY page ORDER BY c DESC LIMIT 10; You may also put this property in `hive-site.xml` to override the default value. @@ -630,22 +628,18 @@ For now, the `mapred.reduce.tasks` property is still recognized, and is converte #### Caching The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no -longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to +longer automatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to let user control table caching explicitly: -``` -CACHE TABLE logs_last_month; -UNCACHE TABLE logs_last_month; -``` + CACHE TABLE logs_last_month; + UNCACHE TABLE logs_last_month; -**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", +**NOTE:** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: -``` -CACHE TABLE logs_last_month; -SELECT COUNT(1) FROM logs_last_month; -``` + CACHE TABLE logs_last_month; + SELECT COUNT(1) FROM logs_last_month; Several caching related features are not supported yet: @@ -655,7 +649,7 @@ Several caching related features are not supported yet: ### Compatibility with Apache Hive -#### Deploying in Exising Hive Warehouses +#### Deploying in Existing Hive Warehouses Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement @@ -666,50 +660,50 @@ or partitioning of your tables. Spark SQL supports the vast majority of Hive features, such as: * Hive query statements, including: - * `SELECT` - * `GROUP BY - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` * All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathemtatical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) * User defined functions (UDF) * User defined aggregation functions (UDAF) -* User defined serialization formats (SerDe's) +* User defined serialization formats (SerDes) * Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` * Unions -* Sub queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` * Sampling * Explain * Partitioned tables * All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` * Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` #### Unsupported Hive Functionality @@ -749,8 +743,7 @@ releases of Spark SQL. Hive automatically converts the join into a map join. We are adding this auto conversion in the next release. * Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "SET - spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". We are going to add auto-setting of parallelism in the next release. * Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still launches tasks to compute the result. From cf049efdc7caeb2577a88f5fa3e7f1b665635451 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 29 Aug 2014 15:29:43 -0700 Subject: [PATCH 177/192] [SPARK-3234][Build] Fixed environment variables that rely on deprecated command line options in make-distribution.sh Please refer to [SPARK-3234](https://issues.apache.org/jira/browse/SPARK-3234) for details. Author: Cheng Lian Closes #2208 from liancheng/spark-3234 and squashes the following commits: fb26de8 [Cheng Lian] Fixed SPARK-3234 (cherry picked from commit 287c0ac7722dd4bc51b921ccc6f0e3c1625b5ff4) Signed-off-by: Patrick Wendell --- make-distribution.sh | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index f7a6a9d838bb6..ee1399071112d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -113,7 +113,17 @@ if ! which mvn &>/dev/null; then echo -e "Download Maven from https://maven.apache.org/" exit -1; fi + VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ + | grep -v "INFO"\ + | tail -n 1) +SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles $@ 2>/dev/null\ + | grep -v "INFO"\ + | fgrep --count "hive";\ + # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\ + # because we use "set -o pipefail" + echo -n) JAVA_CMD="$JAVA_HOME"/bin/java JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) @@ -175,7 +185,7 @@ cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" mkdir -p "$DISTDIR/examples/src/main" cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/" -if [ "$SPARK_HIVE" == "true" ]; then +if [ "$SPARK_HIVE" == "1" ]; then cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi From 9bae345ab543b686e8b32e504711140950d9ea68 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 29 Aug 2014 15:36:04 -0700 Subject: [PATCH 178/192] [SPARK-3269][SQL] Decreases initial buffer size for row set to prevent OOM When a large batch size is specified, `SparkSQLOperationManager` OOMs even if the whole result set is much smaller than the batch size. Author: Cheng Lian Closes #2171 from liancheng/jdbc-fetch-size and squashes the following commits: 5e1623b [Cheng Lian] Decreases initial buffer size for row set to prevent OOM (cherry picked from commit d94a44d7caaf3fe7559d9ad7b10872fa16cf81ca) Signed-off-by: Michael Armbrust --- .../hive/thriftserver/server/SparkSQLOperationManager.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 6eccb1ba6d4dc..f12b5a69a09f7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -66,9 +66,10 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage if (!iter.hasNext) { new RowSet() } else { - val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. + // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int + val maxRows = maxRowsL.toInt var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows) + var rowSet = new ArrayBuffer[Row](maxRows.min(1024)) while (curRow < maxRows && iter.hasNext) { val sparkRow = iter.next() From c1333b85292636c8db24ca08401bd559ed2c71f9 Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Fri, 29 Aug 2014 15:37:43 -0700 Subject: [PATCH 179/192] [SPARK-3291][SQL]TestcaseName in createQueryTest should not contain ":" ":" is not allowed to appear in a file name of Windows system. If file name contains ":", this file can't be checked out in a Windows system and developers using Windows must be careful to not commit the deletion of such files, Which is very inconvenient. Author: qiping.lqp Closes #2191 from chouqin/querytest and squashes the following commits: 0e943a1 [qiping.lqp] rename golden file 60a863f [qiping.lqp] TestcaseName in createQueryTest should not contain ":" (cherry picked from commit 634d04b87c2744d645e9c26e746ba2006371d9b5) Signed-off-by: Michael Armbrust --- ...y when query Hive table-0-5d14d21a239daa42b086cc895215009a} | 0 .../apache/spark/sql/hive/execution/HiveComparisonTest.scala | 3 +++ .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 +- 3 files changed, 4 insertions(+), 1 deletion(-) rename sql/hive/src/test/resources/golden/{case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a => case sensitivity when query Hive table-0-5d14d21a239daa42b086cc895215009a} (100%) diff --git a/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a b/sql/hive/src/test/resources/golden/case sensitivity when query Hive table-0-5d14d21a239daa42b086cc895215009a similarity index 100% rename from sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a rename to sql/hive/src/test/resources/golden/case sensitivity when query Hive table-0-5d14d21a239daa42b086cc895215009a diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 502ce8fb297e9..671c3b162f875 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -195,6 +195,9 @@ abstract class HiveComparisonTest val installHooksCommand = "(?i)SET.*hooks".r def createQueryTest(testCaseName: String, sql: String, reset: Boolean = true) { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + // If test sharding is enable, skip tests that are not in the correct shard. shardInfo.foreach { case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6d925e56e6838..c4abb3eb4861f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -309,7 +309,7 @@ class HiveQuerySuite extends HiveComparisonTest { } } - createQueryTest("case sensitivity: Hive table", + createQueryTest("case sensitivity when query Hive table", "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") test("case sensitivity: registered table") { From 926f171c90d9c3cb83e99c31bccf79c21bd6f5e0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 15:55:26 -0700 Subject: [PATCH 180/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit a4a7a241441489a0d31365e18476ae2e1c34464d. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..c7dc7efc6eb7d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc2 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml From c4b7ec8dd8d63255451acf7e62383217f1f63bdc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 15:55:30 -0700 Subject: [PATCH 181/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc2" This reverts commit 711aebb329ca28046396af1e34395a0df92b5327. --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 +++++---- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 +++-- sql/core/pom.xml | 5 +++-- sql/hive-thriftserver/pom.xml | 5 +++-- sql/hive/pom.xml | 5 +++-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 38 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..a54b34235dfb4 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index c7dc7efc6eb7d..c5a9b3d5f4fd0 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc2 + HEAD @@ -880,7 +881,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From b0facb590eac032f82cea35982c3ed335f2ebbf4 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 29 Aug 2014 17:37:15 -0700 Subject: [PATCH 182/192] [SPARK-3296][mllib] spark-example should be run-example in head notation of DenseKMeans and SparseNaiveBayes `./bin/spark-example` should be `./bin/run-example` in DenseKMeans and SparseNaiveBayes Author: wangfei Closes #2193 from scwf/run-example and squashes the following commits: 207eb3a [wangfei] spark-example should be run-example 27a8999 [wangfei] ./bin/spark-example should be ./bin/run-example (cherry picked from commit 13901764f4e9ed3de03e420d88ab42bdce5d5140) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/examples/mllib/DenseKMeans.scala | 2 +- .../org/apache/spark/examples/mllib/SparseNaiveBayes.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 index f96bc1bf00b92..89dfa26c2299c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.linalg.Vectors /** * An example k-means app. Run with * {{{ - * ./bin/spark-example org.apache.spark.examples.mllib.DenseKMeans [options] + * ./bin/run-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. */ 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 index 88acd9dbb0878..952fa2a5109a4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.util.MLUtils /** * An example naive Bayes app. Run with * {{{ - * ./bin/spark-example org.apache.spark.examples.mllib.SparseNaiveBayes [options] + * ./bin/run-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. */ From aa9364a03ddc793fd2c94981fb168ef8100a507c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 29 Aug 2014 18:16:47 -0700 Subject: [PATCH 183/192] [SPARK-3320][SQL] Made batched in-memory column buffer building work for SchemaRDDs with empty partitions Author: Cheng Lian Closes #2213 from liancheng/spark-3320 and squashes the following commits: 45a0139 [Cheng Lian] Fixed typo in InMemoryColumnarQuerySuite f67067d [Cheng Lian] Fixed SPARK-3320 (cherry picked from commit 32b18dd52cf8920903819f23e406271ecd8ac6bb) Signed-off-by: Michael Armbrust --- .../columnar/InMemoryColumnarTableScan.scala | 49 +++++++------------ .../scala/org/apache/spark/sql/TestData.scala | 5 ++ .../columnar/InMemoryColumnarQuerySuite.scala | 19 +++++-- 3 files changed, 39 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index bc36bacd00b13..cb055cd74a5e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -104,40 +104,29 @@ private[sql] case class InMemoryColumnarTableScan( override def execute() = { relation.cachedColumnBuffers.mapPartitions { iterator => // Find the ordinals of the requested columns. If none are requested, use the first. - val requestedColumns = - if (attributes.isEmpty) { - Seq(0) - } else { - attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) - } - - new Iterator[Row] { - private[this] var columnBuffers: Array[ByteBuffer] = null - private[this] var columnAccessors: Seq[ColumnAccessor] = null - nextBatch() - - private[this] val nextRow = new GenericMutableRow(columnAccessors.length) - - def nextBatch() = { - columnBuffers = iterator.next() - columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) - } + val requestedColumns = if (attributes.isEmpty) { + Seq(0) + } else { + attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + } - override def next() = { - if (!columnAccessors.head.hasNext) { - nextBatch() - } + iterator + .map(batch => requestedColumns.map(batch(_)).map(ColumnAccessor(_))) + .flatMap { columnAccessors => + val nextRow = new GenericMutableRow(columnAccessors.length) + new Iterator[Row] { + override def next() = { + var i = 0 + while (i < nextRow.length) { + columnAccessors(i).extractTo(nextRow, i) + i += 1 + } + nextRow + } - var i = 0 - while (i < nextRow.length) { - columnAccessors(i).extractTo(nextRow, i) - i += 1 + override def hasNext = columnAccessors.head.hasNext } - nextRow } - - override def hasNext = columnAccessors.head.hasNext || iterator.hasNext - } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index c3ec82fb69778..eb33a61c6e811 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -151,4 +151,9 @@ object TestData { TimestampField(new Timestamp(i)) }) timestamps.registerTempTable("timestamps") + + case class IntField(i: Int) + // An RDD with 4 elements and 8 partitions + val withEmptyParts = TestSQLContext.sparkContext.parallelize((1 to 4).map(IntField), 8) + withEmptyParts.registerTempTable("withEmptyParts") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index fdd2799a53268..0e3c67f5eed29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.columnar -import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{SQLConf, QueryTest, TestData} class InMemoryColumnarQuerySuite extends QueryTest { - import TestData._ - import TestSQLContext._ + import org.apache.spark.sql.TestData._ + import org.apache.spark.sql.test.TestSQLContext._ test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan @@ -93,4 +92,16 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT time FROM timestamps"), timestamps.collect().toSeq) } + + test("SPARK-3320 regression: batched column buffer building should work with empty partitions") { + checkAnswer( + sql("SELECT * FROM withEmptyParts"), + withEmptyParts.collect().toSeq) + + TestSQLContext.cacheTable("withEmptyParts") + + checkAnswer( + sql("SELECT * FROM withEmptyParts"), + withEmptyParts.collect().toSeq) + } } From 272b4a68d9b4ae2d74df7615122c587bfac2dddf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 18:49:51 -0700 Subject: [PATCH 184/192] Adding new CHANGES.txt --- CHANGES.txt | 45 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index b42721ac45bfe..3693d59947ce7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,51 @@ Spark Change Log Release 1.1.0 + [SPARK-3320][SQL] Made batched in-memory column buffer building work for SchemaRDDs with empty partitions + Cheng Lian + 2014-08-29 18:16:47 -0700 + Commit: aa9364a, github.com/apache/spark/pull/2213 + + [SPARK-3296][mllib] spark-example should be run-example in head notation of DenseKMeans and SparseNaiveBayes + wangfei + 2014-08-29 17:37:15 -0700 + Commit: b0facb5, github.com/apache/spark/pull/2193 + + [SPARK-3291][SQL]TestcaseName in createQueryTest should not contain ":" + qiping.lqp + 2014-08-29 15:37:43 -0700 + Commit: c1333b8, github.com/apache/spark/pull/2191 + + [SPARK-3269][SQL] Decreases initial buffer size for row set to prevent OOM + Cheng Lian + 2014-08-29 15:36:04 -0700 + Commit: 9bae345, github.com/apache/spark/pull/2171 + + [SPARK-3234][Build] Fixed environment variables that rely on deprecated command line options in make-distribution.sh + Cheng Lian + 2014-08-29 15:29:43 -0700 + Commit: cf049ef, github.com/apache/spark/pull/2208 + + [Docs] SQL doc formatting and typo fixes + Nicholas Chammas , nchammas + 2014-08-29 15:23:32 -0700 + Commit: bfa2dc9, github.com/apache/spark/pull/2201 + + [SPARK-3307] [PySpark] Fix doc string of SparkContext.broadcast() + Davies Liu + 2014-08-29 11:47:49 -0700 + Commit: 98d0716, github.com/apache/spark/pull/2202 + + HOTFIX: Bump spark-ec2 version to 1.1.0 + Patrick Wendell + 2014-08-29 11:20:45 -0700 + Commit: c71b5c6 + + Adding new CHANGES.txt + Patrick Wendell + 2014-08-28 17:17:30 -0700 + Commit: 7db87b3 + [SPARK-3277] Fix external spilling with LZ4 assertion error Andrew Or , Patrick Wendell 2014-08-28 17:05:21 -0700 From 2b2e02265f80e4c5172c1e498aa9ba2c6b91c6c9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 02:26:03 +0000 Subject: [PATCH 185/192] [maven-release-plugin] prepare release v1.1.0-rc3 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index c5a9b3d5f4fd0..67360d0864f8b 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc3 @@ -881,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From 8b5f0dbd8d32a25a4e7ba3ebe1a4c3c6310aeb85 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 02:26:11 +0000 Subject: [PATCH 186/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 67360d0864f8b..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc3 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From d4ce26426832eb09a229c51cb5e64048272cf4b6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 29 Aug 2014 22:24:35 -0700 Subject: [PATCH 187/192] BUILD: Adding back CDH4 as per user requests --- dev/create-release/create-release.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 7549fbbe66654..281e8d4de6d71 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,6 +118,7 @@ make_binary_release() { } make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Pyarn" & make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Pyarn" & make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & From 829025e2314554c6dea01abcff30caf9bbc2e7d6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 10:14:28 -0700 Subject: [PATCH 188/192] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 8b5f0dbd8d32a25a4e7ba3ebe1a4c3c6310aeb85. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4709b7dbddfea..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index f29540b239c73..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index debc4dd703d9a..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index f35d3d6a788e3..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 17d0fe233873f..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 402af353152d8..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 5123d0554639c..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9c00bfc8429a4..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 1b9ef4af0c2ed..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 60292a2683212..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 58b995c5e7005..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 02c9676fb086a..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 656478583fac2..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 74f528f030987..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 2a2e1e35a09c5..67360d0864f8b 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc3 diff --git a/repl/pom.xml b/repl/pom.xml index 8748ada36f57a..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e2356381c07fb..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3efea9ab8b247..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c264ff4ec92e5..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1e689e6d6dcf2..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c0ce0d7c7478d..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c601fd5fbbee2..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 18f27b827ff1a..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 2ba3baf0e3b2e..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.1-SNAPSHOT + 1.1.0 ../pom.xml From d9a1c96f727a424f6b8a6dbc3f29d32e27f52848 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 10:14:33 -0700 Subject: [PATCH 189/192] Revert "[maven-release-plugin] prepare release v1.1.0-rc3" This reverts commit 2b2e02265f80e4c5172c1e498aa9ba2c6b91c6c9. --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 +++++---- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 +++-- sql/core/pom.xml | 5 +++-- sql/hive-thriftserver/pom.xml | 5 +++-- sql/hive/pom.xml | 5 +++-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 38 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..a54b34235dfb4 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 67360d0864f8b..c5a9b3d5f4fd0 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -25,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +41,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc3 + HEAD @@ -880,7 +881,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c6f60c18804a4 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,12 +16,13 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.0-SNAPSHOT ../pom.xml From b2d0493b223c5f98a593bb6d7372706cc02bebad Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 17:48:02 +0000 Subject: [PATCH 190/192] [maven-release-plugin] prepare release v1.1.0-rc3 --- assembly/pom.xml | 6 +++--- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 9 ++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 5 ++--- sql/core/pom.xml | 5 ++--- sql/hive-thriftserver/pom.xml | 5 ++--- sql/hive/pom.xml | 5 ++--- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 33 insertions(+), 38 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..799f8d9b3c815 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml @@ -124,8 +124,8 @@ log4j.properties - - + + diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..8eec7e5dd23b5 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83e60268afbd2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9bde90ed116e1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..daaae05a3dace 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..830eb32c32c0c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..e3df55355c8d5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..1f9e52b19b24e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..d28741a9524dc 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e7609985d2c0 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..4d0eac8956955 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..44f50aeefa62e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..c677f361cf1f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..873f65105b1c5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/pom.xml b/pom.xml index c5a9b3d5f4fd0..67360d0864f8b 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -26,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 pom Spark Project Parent POM http://spark.apache.org/ @@ -41,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - HEAD + v1.1.0-rc3 @@ -881,7 +880,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - + true ${session.executionRootDirectory} diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..b7458eeb270dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..7c9e5b284e0d9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..d797753f12151 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..d75d2e514544d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..dc5cfbac8212d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -16,13 +16,12 @@ ~ limitations under the License. --> - + 4.0.0 org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..31c096380a7c1 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..ce3629443ed98 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..274be3a563641 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..64fb00ac71b60 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.1.0 ../pom.xml From 865e6f63f63f5e881a02d1a4e3b4c5d0e86fcd8e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 30 Aug 2014 17:48:10 +0000 Subject: [PATCH 191/192] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 24 files changed, 25 insertions(+), 25 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 799f8d9b3c815..4709b7dbddfea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8eec7e5dd23b5..f29540b239c73 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 83e60268afbd2..debc4dd703d9a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9bde90ed116e1..f35d3d6a788e3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index daaae05a3dace..17d0fe233873f 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 830eb32c32c0c..402af353152d8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index e3df55355c8d5..5123d0554639c 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 1f9e52b19b24e..9c00bfc8429a4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index d28741a9524dc..1b9ef4af0c2ed 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 7e7609985d2c0..60292a2683212 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 4d0eac8956955..58b995c5e7005 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 44f50aeefa62e..02c9676fb086a 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c677f361cf1f2..656478583fac2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 873f65105b1c5..74f528f030987 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 67360d0864f8b..2a2e1e35a09c5 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/spark.git scm:git:https://git-wip-us.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git - v1.1.0-rc3 + HEAD diff --git a/repl/pom.xml b/repl/pom.xml index b7458eeb270dd..8748ada36f57a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7c9e5b284e0d9..e2356381c07fb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d797753f12151..3efea9ab8b247 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d75d2e514544d..c264ff4ec92e5 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index dc5cfbac8212d..1e689e6d6dcf2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 31c096380a7c1..c0ce0d7c7478d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ce3629443ed98..c601fd5fbbee2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 274be3a563641..18f27b827ff1a 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 64fb00ac71b60..2ba3baf0e3b2e 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0 + 1.1.1-SNAPSHOT ../pom.xml From e136312913517f0e2cbce3742076e955fc237c42 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Mon, 1 Sep 2014 22:14:28 -0700 Subject: [PATCH 192/192] [SPARK-3342] Add SSDs to block device mapping On `m3.2xlarge` instances the 2x80GB SSDs are inaccessible if not added to the block device mapping when the instance is created. They work when added with this patch. I have not tested this with other instance types, and I do not know much about this script and EC2 deployment in general. Maybe this code needs to depend on the instance type. The requirement for this mapping is described in the AWS docs at: http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html#InstanceStore_UsageScenarios "For M3 instances, you must specify instance store volumes in the block device mapping for the instance. When you launch an M3 instance, we ignore any instance store volumes specified in the block device mapping for the AMI." Author: Daniel Darabos Closes #2081 from darabos/patch-1 and squashes the following commits: 1ceb2c8 [Daniel Darabos] Use %d string interpolation instead of {}. a1854d7 [Daniel Darabos] Only specify ephemeral device mapping for M3. e0d9e37 [Daniel Darabos] Create ephemeral device mapping based on get_num_disks(). 6b116a6 [Daniel Darabos] Add SSDs to block device mapping --- ec2/spark_ec2.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 59356c5dd96ec..e22ca8d117225 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -26,6 +26,7 @@ import pipes import random import shutil +import string import subprocess import sys import tempfile @@ -34,7 +35,7 @@ from optparse import OptionParser from sys import stderr import boto -from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType +from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 # A URL prefix from which to fetch AMI information @@ -341,6 +342,15 @@ def launch_cluster(conn, opts, cluster_name): device.delete_on_termination = True block_map["/dev/sdv"] = device + # AWS ignores the AMI-specified block device mapping for M3 (see SPARK-3342). + if opts.instance_type.startswith('m3.'): + for i in range(get_num_disks(opts.instance_type)): + dev = BlockDeviceType() + dev.ephemeral_name = 'ephemeral%d' % i + # The first ephemeral drive is /dev/sdb. + name = '/dev/sd' + string.letters[i + 1] + block_map[name] = dev + # Launch slaves if opts.spot_price is not None: # Launch spot instances with the requested price