From 4c589cac4496c6a4bb8485a340bd0641dca13847 Mon Sep 17 00:00:00 2001 From: Shiti Date: Thu, 16 Oct 2014 10:52:06 -0700 Subject: [PATCH 01/32] [SPARK-3944][Core] Code re-factored as suggested Author: Shiti Closes #2810 from Shiti/master and squashes the following commits: 051d82f [Shiti] setting the default value of uri scheme to "file" where matching "file" or None yields the same result --- .../main/scala/org/apache/spark/util/Utils.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index cbc4095065a19..53a7512edd852 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -340,8 +340,8 @@ private[spark] object Utils extends Logging { val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) - Option(uri.getScheme) match { - case Some("http") | Some("https") | Some("ftp") => + Option(uri.getScheme).getOrElse("file") match { + case "http" | "https" | "ftp" => logInfo("Fetching " + url + " to " + tempFile) var uc: URLConnection = null @@ -374,7 +374,7 @@ private[spark] object Utils extends Logging { } } Files.move(tempFile, targetFile) - case Some("file") | None => + case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) @@ -403,7 +403,7 @@ private[spark] object Utils extends Logging { logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) Files.copy(sourceFile, targetFile) } - case Some(other) => + case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) @@ -1401,10 +1401,10 @@ private[spark] object Utils extends Logging { paths.split(",").filter { p => val formattedPath = if (windows) formatWindowsPath(p) else p val uri = new URI(formattedPath) - Option(uri.getScheme) match { - case Some(windowsDrive(d)) if windows => false - case Some("local") | Some("file") | None => false - case Some(other) => true + Option(uri.getScheme).getOrElse("file") match { + case windowsDrive(d) if windows => false + case "local" | "file" => false + case _ => true } } } From 091d32c52e9d73da95896016c1d920e89858abfa Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Oct 2014 14:56:50 -0700 Subject: [PATCH 02/32] [SPARK-3971] [MLLib] [PySpark] hotfix: Customized pickler should work in cluster mode Customized pickler should be registered before unpickling, but in executor, there is no way to register the picklers before run the tasks. So, we need to register the picklers in the tasks itself, duplicate the javaToPython() and pythonToJava() in MLlib, call SerDe.initialize() before pickling or unpickling. Author: Davies Liu Closes #2830 from davies/fix_pickle and squashes the following commits: 0c85fb9 [Davies Liu] revert the privacy change 6b94e15 [Davies Liu] use JavaConverters instead of JavaConversions 0f02050 [Davies Liu] hotfix: Customized pickler does not work in cluster --- .../apache/spark/api/python/PythonRDD.scala | 7 ++- .../apache/spark/api/python/SerDeUtil.scala | 14 ++++- .../mllib/api/python/PythonMLLibAPI.scala | 52 +++++++++++++++++-- python/pyspark/context.py | 2 - python/pyspark/mllib/classification.py | 4 +- python/pyspark/mllib/clustering.py | 4 +- python/pyspark/mllib/feature.py | 5 +- python/pyspark/mllib/linalg.py | 13 +++++ python/pyspark/mllib/random.py | 2 +- python/pyspark/mllib/recommendation.py | 7 +-- python/pyspark/mllib/regression.py | 4 +- python/pyspark/mllib/stat.py | 7 +-- python/pyspark/mllib/tree.py | 8 +-- python/pyspark/mllib/util.py | 6 +-- 14 files changed, 101 insertions(+), 34 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 4acbdf9d5e25f..29ca751519abd 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 @@ -23,6 +23,7 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials @@ -746,6 +747,7 @@ private[spark] object PythonRDD extends Logging { def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { pyRDD.rdd.mapPartitions { iter => val unpickle = new Unpickler + SerDeUtil.initialize() iter.flatMap { row => unpickle.loads(row) match { // in case of objects are pickled in batch mode @@ -785,7 +787,7 @@ private[spark] object PythonRDD extends Logging { }.toJavaRDD() } - private class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private[spark] class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { private val pickle = new Pickler() private var batch = 1 private val buffer = new mutable.ArrayBuffer[Any] @@ -822,11 +824,12 @@ private[spark] object PythonRDD extends Logging { */ def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { pyRDD.rdd.mapPartitions { iter => + SerDeUtil.initialize() val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) if (batched) { - obj.asInstanceOf[JArrayList[_]] + obj.asInstanceOf[JArrayList[_]].asScala } else { Seq(obj) } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 7903457b17e13..ebdc3533e0992 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -29,7 +29,7 @@ import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ -private[python] object SerDeUtil extends Logging { +private[spark] object SerDeUtil extends Logging { // Unpickle array.array generated by Python 2.6 class ArrayConstructor extends net.razorvine.pickle.objects.ArrayConstructor { // /* Description of types */ @@ -76,9 +76,18 @@ private[python] object SerDeUtil extends Logging { } } + private var initialized = false + // This should be called before trying to unpickle array.array from Python + // In cluster mode, this should be put in closure def initialize() = { - Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + synchronized{ + if (!initialized) { + Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + initialized = true + } + } } + initialize() private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { val pickle = new Pickler @@ -143,6 +152,7 @@ private[python] object SerDeUtil extends Logging { obj.asInstanceOf[Array[_]].length == 2 } pyRDD.mapPartitions { iter => + initialize() val unpickle = new Unpickler val unpickled = if (batchSerialized) { 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 f7251e65e04f1..9a100170b75c6 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream +import java.util.{ArrayList => JArrayList} import scala.collection.JavaConverters._ import scala.language.existentials @@ -27,6 +28,7 @@ import net.razorvine.pickle._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.feature.Word2Vec @@ -639,13 +641,24 @@ private[spark] object SerDe extends Serializable { } } + var initialized = false + // This should be called before trying to serialize any above classes + // In cluster mode, this should be put in the closure def initialize(): Unit = { - new DenseVectorPickler().register() - new DenseMatrixPickler().register() - new SparseVectorPickler().register() - new LabeledPointPickler().register() - new RatingPickler().register() + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseVectorPickler().register() + new LabeledPointPickler().register() + new RatingPickler().register() + initialized = true + } + } } + // will not called in Executor automatically + initialize() def dumps(obj: AnyRef): Array[Byte] = { new Pickler().dumps(obj) @@ -659,4 +672,33 @@ private[spark] object SerDe extends Serializable { def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + new PythonRDD.AutoBatchedPickler(iter) + } + } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]].asScala + } else { + Seq(obj) + } + } + }.toJavaRDD() + } } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 89d2e2e5b4a8e..8d27ccb95f82c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -215,8 +215,6 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile - SparkContext._jvm.SerDeUtil.initialize() - SparkContext._jvm.SerDe.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index cd43982191702..e295c9d0954d9 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -21,7 +21,7 @@ from numpy import array from pyspark import SparkContext, PickleSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -244,7 +244,7 @@ def train(cls, data, lambda_=1.0): :param lambda_: The smoothing parameter """ sc = data.context - jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) + jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(_to_java_object_rdd(data), lambda_) labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 12c56022717a5..5ee7997104d21 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -17,7 +17,7 @@ from pyspark import SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd __all__ = ['KMeansModel', 'KMeans'] @@ -85,7 +85,7 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" # cache serialized data to avoid objects over head in JVM cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() model = sc._jvm.PythonMLLibAPI().trainKMeansModel( - cached._to_java_object_rdd(), k, maxIterations, runs, initializationMode) + _to_java_object_rdd(cached), k, maxIterations, runs, initializationMode) bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) centers = ser.loads(str(bytes)) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index f4cbf31b94fe2..b5a3f22c6907e 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -19,8 +19,7 @@ Python package for feature in MLlib. """ from pyspark.serializers import PickleSerializer, AutoBatchedSerializer - -from pyspark.mllib.linalg import _convert_to_vector +from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd __all__ = ['Word2Vec', 'Word2VecModel'] @@ -176,7 +175,7 @@ def fit(self, data): seed = self.seed model = sc._jvm.PythonMLLibAPI().trainWord2Vec( - data._to_java_object_rdd(), vectorSize, + _to_java_object_rdd(data), vectorSize, learningRate, numPartitions, numIterations, seed) return Word2VecModel(sc, model) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 24c5480b2f753..773d8d393805d 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,6 +29,8 @@ import numpy as np +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer + __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] @@ -50,6 +52,17 @@ def fast_pickle_array(ar): _have_scipy = False +# this will call the MLlib version of pythonToJava() +def _to_java_object_rdd(rdd): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) + return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) + + def _convert_to_vector(l): if isinstance(l, Vector): return l diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index a787e4dea2c55..73baba4ace5f6 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -32,7 +32,7 @@ def serialize(f): @wraps(f) def func(sc, *a, **kw): jrdd = f(sc, *a, **kw) - return RDD(sc._jvm.PythonRDD.javaToPython(jrdd), sc, + return RDD(sc._jvm.SerDe.javaToPython(jrdd), sc, BatchedSerializer(PickleSerializer(), 1024)) return func diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 59c1c5ff0ced0..17f96b8700bd7 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -18,6 +18,7 @@ from pyspark import SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD +from pyspark.mllib.linalg import _to_java_object_rdd __all__ = ['MatrixFactorizationModel', 'ALS'] @@ -77,9 +78,9 @@ def predictAll(self, user_product): first = tuple(map(int, first)) assert all(type(x) is int for x in first), "user and product in user_product shoul be int" sc = self._context - tuplerdd = sc._jvm.SerDe.asTupleRDD(user_product._to_java_object_rdd().rdd()) + tuplerdd = sc._jvm.SerDe.asTupleRDD(_to_java_object_rdd(user_product).rdd()) jresult = self._java_model.predict(tuplerdd).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(jresult), sc, + return RDD(sc._jvm.SerDe.javaToPython(jresult), sc, AutoBatchedSerializer(PickleSerializer())) @@ -97,7 +98,7 @@ def _prepare(cls, ratings): # serialize them by AutoBatchedSerializer before cache to reduce the # objects overhead in JVM cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() - return cached._to_java_object_rdd() + return _to_java_object_rdd(cached) @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 12b322aaae796..93e17faf5cd51 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -19,8 +19,8 @@ from numpy import array from pyspark import SparkContext -from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -131,7 +131,7 @@ def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights) # use AutoBatchedSerializer before cache to reduce the memory # overhead in JVM cached = data._reserialize(AutoBatchedSerializer(ser)).cache() - ans = train_func(cached._to_java_object_rdd(), initial_bytes) + ans = train_func(_to_java_object_rdd(cached), initial_bytes) assert len(ans) == 2, "JVM call result had unexpected length" weights = ser.loads(str(ans[0])) return modelClass(weights, ans[1]) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index b9de0909a6fb1..a6019dadf781c 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -22,6 +22,7 @@ from functools import wraps from pyspark import PickleSerializer +from pyspark.mllib.linalg import _to_java_object_rdd __all__ = ['MultivariateStatisticalSummary', 'Statistics'] @@ -106,7 +107,7 @@ def colStats(rdd): array([ 2., 0., 0., -2.]) """ sc = rdd.ctx - jrdd = rdd._to_java_object_rdd() + jrdd = _to_java_object_rdd(rdd) cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @@ -162,14 +163,14 @@ def corr(x, y=None, method=None): if type(y) == str: raise TypeError("Use 'method=' to specify method name.") - jx = x._to_java_object_rdd() + jx = _to_java_object_rdd(x) if not y: resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) bytes = sc._jvm.SerDe.dumps(resultMat) ser = PickleSerializer() return ser.loads(str(bytes)).toArray() else: - jy = y._to_java_object_rdd() + jy = _to_java_object_rdd(y) return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 5d7abfb96b7fe..0938eebd3a548 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -19,7 +19,7 @@ from pyspark import SparkContext, RDD from pyspark.serializers import BatchedSerializer, PickleSerializer -from pyspark.mllib.linalg import Vector, _convert_to_vector +from pyspark.mllib.linalg import Vector, _convert_to_vector, _to_java_object_rdd from pyspark.mllib.regression import LabeledPoint __all__ = ['DecisionTreeModel', 'DecisionTree'] @@ -61,8 +61,8 @@ def predict(self, x): return self._sc.parallelize([]) if not isinstance(first[0], Vector): x = x.map(_convert_to_vector) - jPred = self._java_model.predict(x._to_java_object_rdd()).toJavaRDD() - jpyrdd = self._sc._jvm.PythonRDD.javaToPython(jPred) + jPred = self._java_model.predict(_to_java_object_rdd(x)).toJavaRDD() + jpyrdd = self._sc._jvm.SerDe.javaToPython(jPred) return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) else: @@ -104,7 +104,7 @@ def _train(data, type, numClasses, categoricalFeaturesInfo, first = data.first() assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" sc = data.context - jrdd = data._to_java_object_rdd() + jrdd = _to_java_object_rdd(data) cfiMap = MapConverter().convert(categoricalFeaturesInfo, sc._gateway._gateway_client) model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 1357fd4fbc8aa..84b39a48619d2 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -19,7 +19,7 @@ import warnings from pyspark.rdd import RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -174,8 +174,8 @@ def loadLabeledPoints(sc, path, minPartitions=None): """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - jpyrdd = sc._jvm.PythonRDD.javaToPython(jrdd) - return RDD(jpyrdd, sc, BatchedSerializer(PickleSerializer())) + jpyrdd = sc._jvm.SerDe.javaToPython(jrdd) + return RDD(jpyrdd, sc, AutoBatchedSerializer(PickleSerializer())) def _test(): From 99e416b6d64402a5432a265797a1c155a38f4e6f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 16 Oct 2014 16:15:55 -0700 Subject: [PATCH 03/32] [SQL] Fixes the race condition that may cause test failure The removed `Future` was used to end the test case as soon as the Spark SQL CLI process exits. When the process exits prematurely, this mechanism prevents the test case to wait until timeout. But it also creates a race condition: when `foundAllExpectedAnswers.tryFailure` is called, there are chances that the last expected output line of the CLI process hasn't been caught by the main logics of the test code, thus fails the test case. Removing this `Future` doesn't affect correctness. Author: Cheng Lian Closes #2823 from liancheng/clean-clisuite and squashes the following commits: 489a97c [Cheng Lian] Fixes the race condition that may cause test failure --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 6 ------ 1 file changed, 6 deletions(-) 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 fc97a25be34be..8a72e9d2aef57 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 @@ -78,12 +78,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val process = (Process(command) #< queryStream).run( ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - Future { - val exitValue = process.exitValue() - foundAllExpectedAnswers.tryFailure( - new SparkException(s"Spark SQL CLI process exit value: $exitValue")) - } - try { Await.result(foundAllExpectedAnswers.future, timeout) } catch { case cause: Throwable => From 2fe0ba95616bb3860736b6b426635a5d2a0e9bd9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 16 Oct 2014 21:38:45 -0400 Subject: [PATCH 04/32] SPARK-3874: Provide stable TaskContext API This is a small number of clean-up changes on top of #2782. Closes #2782. Author: Prashant Sharma Author: Patrick Wendell Closes #2803 from pwendell/pr-2782 and squashes the following commits: 56d5b7a [Patrick Wendell] Minor clean-up 44089ec [Patrick Wendell] Clean-up the TaskContext API. ed551ce [Prashant Sharma] Fixed a typo df261d0 [Prashant Sharma] Josh's suggestion facf3b1 [Prashant Sharma] Fixed the mima issue. 7ecc2fe [Prashant Sharma] CR, Moved implementations to TaskContextImpl bbd9e05 [Prashant Sharma] adding missed out files to git. ef633f5 [Prashant Sharma] SPARK-3874, Provide stable TaskContext API --- .../java/org/apache/spark/TaskContext.java | 225 +++--------------- .../org/apache/spark/TaskContextHelper.scala | 29 +++ .../org/apache/spark/TaskContextImpl.scala | 91 +++++++ .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 8 +- .../apache/spark/scheduler/DAGScheduler.scala | 6 +- .../org/apache/spark/scheduler/Task.scala | 10 +- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../util/JavaTaskCompletionListenerImpl.java | 4 +- .../org/apache/spark/CacheManagerSuite.scala | 8 +- .../org/apache/spark/rdd/PipedRDDSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 8 +- project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 6 +- .../sql/parquet/ParquetTableOperations.scala | 4 +- 16 files changed, 186 insertions(+), 223 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/TaskContextHelper.scala create mode 100644 core/src/main/scala/org/apache/spark/TaskContextImpl.scala diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 4e6d708af0ea7..2d998d4c7a5d9 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -18,131 +18,55 @@ package org.apache.spark; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; import scala.Function0; import scala.Function1; import scala.Unit; -import scala.collection.JavaConversions; import org.apache.spark.annotation.DeveloperApi; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.util.TaskCompletionListener; -import org.apache.spark.util.TaskCompletionListenerException; /** -* :: DeveloperApi :: -* Contextual information about a task which can be read or mutated during execution. -*/ -@DeveloperApi -public class TaskContext implements Serializable { - - private int stageId; - private int partitionId; - private long attemptId; - private boolean runningLocally; - private TaskMetrics taskMetrics; - - /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - * @param taskMetrics performance metrics of the task - */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally, - TaskMetrics taskMetrics) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = runningLocally; - this.stageId = stageId; - this.taskMetrics = taskMetrics; - } - - /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = runningLocally; - this.stageId = stageId; - this.taskMetrics = TaskMetrics.empty(); - } - + * Contextual information about a task which can be read or mutated during + * execution. To access the TaskContext for a running task use + * TaskContext.get(). + */ +public abstract class TaskContext implements Serializable { /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task + * Return the currently active TaskContext. This can be called inside of + * user functions to access contextual information about running tasks. */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = false; - this.stageId = stageId; - this.taskMetrics = TaskMetrics.empty(); + public static TaskContext get() { + return taskContext.get(); } private static ThreadLocal taskContext = new ThreadLocal(); - /** - * :: Internal API :: - * This is spark internal API, not intended to be called from user programs. - */ - public static void setTaskContext(TaskContext tc) { + static void setTaskContext(TaskContext tc) { taskContext.set(tc); } - public static TaskContext get() { - return taskContext.get(); - } - - /** :: Internal API :: */ - public static void unset() { + static void unset() { taskContext.remove(); } - // List of callback functions to execute when the task completes. - private transient List onCompleteCallbacks = - new ArrayList(); - - // Whether the corresponding task has been killed. - private volatile boolean interrupted = false; - - // Whether the task has completed. - private volatile boolean completed = false; - /** - * Checks whether the task has completed. + * Whether the task has completed. */ - public boolean isCompleted() { - return completed; - } + public abstract boolean isCompleted(); /** - * Checks whether the task has been killed. + * Whether the task has been killed. */ - public boolean isInterrupted() { - return interrupted; - } + public abstract boolean isInterrupted(); + + /** @deprecated: use isRunningLocally() */ + @Deprecated + public abstract boolean runningLocally(); + + public abstract boolean isRunningLocally(); /** * Add a (Java friendly) listener to be executed on task completion. @@ -150,10 +74,7 @@ public boolean isInterrupted() { *

* An example use is for HadoopRDD to register a callback to close the input stream. */ - public TaskContext addTaskCompletionListener(TaskCompletionListener listener) { - onCompleteCallbacks.add(listener); - return this; - } + public abstract TaskContext addTaskCompletionListener(TaskCompletionListener listener); /** * Add a listener in the form of a Scala closure to be executed on task completion. @@ -161,109 +82,27 @@ public TaskContext addTaskCompletionListener(TaskCompletionListener listener) { *

* An example use is for HadoopRDD to register a callback to close the input stream. */ - public TaskContext addTaskCompletionListener(final Function1 f) { - onCompleteCallbacks.add(new TaskCompletionListener() { - @Override - public void onTaskCompletion(TaskContext context) { - f.apply(context); - } - }); - return this; - } + public abstract TaskContext addTaskCompletionListener(final Function1 f); /** * Add a callback function to be executed on task completion. An example use * is for HadoopRDD to register a callback to close the input stream. * Will be called in any situation - success, failure, or cancellation. * - * Deprecated: use addTaskCompletionListener - * + * @deprecated: use addTaskCompletionListener + * * @param f Callback function. */ @Deprecated - public void addOnCompleteCallback(final Function0 f) { - onCompleteCallbacks.add(new TaskCompletionListener() { - @Override - public void onTaskCompletion(TaskContext context) { - f.apply(); - } - }); - } - - /** - * ::Internal API:: - * Marks the task as completed and triggers the listeners. - */ - public void markTaskCompleted() throws TaskCompletionListenerException { - completed = true; - List errorMsgs = new ArrayList(2); - // Process complete callbacks in the reverse order of registration - List revlist = - new ArrayList(onCompleteCallbacks); - Collections.reverse(revlist); - for (TaskCompletionListener tcl: revlist) { - try { - tcl.onTaskCompletion(this); - } catch (Throwable e) { - errorMsgs.add(e.getMessage()); - } - } - - if (!errorMsgs.isEmpty()) { - throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); - } - } - - /** - * ::Internal API:: - * Marks the task for interruption, i.e. cancellation. - */ - public void markInterrupted() { - interrupted = true; - } - - @Deprecated - /** Deprecated: use getStageId() */ - public int stageId() { - return stageId; - } - - @Deprecated - /** Deprecated: use getPartitionId() */ - public int partitionId() { - return partitionId; - } - - @Deprecated - /** Deprecated: use getAttemptId() */ - public long attemptId() { - return attemptId; - } - - @Deprecated - /** Deprecated: use isRunningLocally() */ - public boolean runningLocally() { - return runningLocally; - } - - public boolean isRunningLocally() { - return runningLocally; - } + public abstract void addOnCompleteCallback(final Function0 f); - public int getStageId() { - return stageId; - } + public abstract int stageId(); - public int getPartitionId() { - return partitionId; - } + public abstract int partitionId(); - public long getAttemptId() { - return attemptId; - } + public abstract long attemptId(); - /** ::Internal API:: */ - public TaskMetrics taskMetrics() { - return taskMetrics; - } + /** ::DeveloperApi:: */ + @DeveloperApi + public abstract TaskMetrics taskMetrics(); } diff --git a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala b/core/src/main/scala/org/apache/spark/TaskContextHelper.scala new file mode 100644 index 0000000000000..4636c4600a01a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskContextHelper.scala @@ -0,0 +1,29 @@ +/* + * 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 + +/** + * This class exists to restrict the visibility of TaskContext setters. + */ +private [spark] object TaskContextHelper { + + def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) + + def unset(): Unit = TaskContext.unset() + +} diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala new file mode 100644 index 0000000000000..afd2b85d33a77 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -0,0 +1,91 @@ +/* + * 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 + +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} + +import scala.collection.mutable.ArrayBuffer + +private[spark] class TaskContextImpl(val stageId: Int, + val partitionId: Int, + val attemptId: Long, + val runningLocally: Boolean = false, + val taskMetrics: TaskMetrics = TaskMetrics.empty) + extends TaskContext + with Logging { + + // List of callback functions to execute when the task completes. + @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] + + // Whether the corresponding task has been killed. + @volatile private var interrupted: Boolean = false + + // Whether the task has completed. + @volatile private var completed: Boolean = false + + override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { + onCompleteCallbacks += listener + this + } + + override def addTaskCompletionListener(f: TaskContext => Unit): this.type = { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f(context) + } + this + } + + @deprecated("use addTaskCompletionListener", "1.1.0") + override def addOnCompleteCallback(f: () => Unit) { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f() + } + } + + /** Marks the task as completed and triggers the listeners. */ + private[spark] def markTaskCompleted(): Unit = { + completed = true + val errorMsgs = new ArrayBuffer[String](2) + // Process complete callbacks in the reverse order of registration + onCompleteCallbacks.reverse.foreach { listener => + try { + listener.onTaskCompletion(this) + } catch { + case e: Throwable => + errorMsgs += e.getMessage + logError("Error in TaskCompletionListener", e) + } + } + if (errorMsgs.nonEmpty) { + throw new TaskCompletionListenerException(errorMsgs) + } + } + + /** Marks the task for interruption, i.e. cancellation. */ + private[spark] def markInterrupted(): Unit = { + interrupted = true + } + + override def isCompleted: Boolean = completed + + override def isRunningLocally: Boolean = runningLocally + + override def isInterrupted: Boolean = interrupted +} + diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 6b63eb23e9ee1..8010dd90082f8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -196,7 +196,7 @@ class HadoopRDD[K, V]( val jobConf = getJobConf() val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.getStageId, theSplit.index, context.getAttemptId.toInt, jobConf) + context.stageId, theSplit.index, context.attemptId.toInt, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 0d97506450a7f..929ded58a3bd5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -956,9 +956,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outfmt.newInstance @@ -1027,9 +1027,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt - writer.setup(context.getStageId, context.getPartitionId, attemptNumber) + writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() try { var count = 0 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 788eb1ff4e455..f81fa6d8089fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -633,14 +633,14 @@ class DAGScheduler( val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) val taskContext = - new TaskContext(job.finalStage.id, job.partitions(0), 0, true) - TaskContext.setTaskContext(taskContext) + new TaskContextImpl(job.finalStage.id, job.partitions(0), 0, true) + TaskContextHelper.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() - TaskContext.unset() + TaskContextHelper.unset() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index c6e47c84a0cb2..2552d03d18d06 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import org.apache.spark.TaskContext +import org.apache.spark.{TaskContextHelper, TaskContextImpl, TaskContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream @@ -45,8 +45,8 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContext(stageId, partitionId, attemptId, false) - TaskContext.setTaskContext(context) + context = new TaskContextImpl(stageId, partitionId, attemptId, false) + TaskContextHelper.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { @@ -56,7 +56,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex runTask(context) } finally { context.markTaskCompleted() - TaskContext.unset() + TaskContextHelper.unset() } } @@ -70,7 +70,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex var metrics: Option[TaskMetrics] = None // Task context, to be initialized in run(). - @transient protected var context: TaskContext = _ + @transient protected var context: TaskContextImpl = _ // The actual Thread on which the task is running, if any. Initialized in run(). @volatile @transient private var taskThread: Thread = _ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 4a078435447e5..b8fa822ae4bd8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -776,7 +776,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0L, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java index 0944bf8cd5c71..e9ec700e32e15 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java @@ -30,8 +30,8 @@ public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { public void onTaskCompletion(TaskContext context) { context.isCompleted(); context.isInterrupted(); - context.getStageId(); - context.getPartitionId(); + context.stageId(); + context.partitionId(); context.isRunningLocally(); context.addTaskCompletionListener(this); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index d735010d7c9d5..c0735f448d193 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -66,7 +66,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) @@ -81,7 +81,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, true) + val context = new TaskContextImpl(0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -102,7 +102,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index be972c5e97a7e..271a90c6646bb 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -174,7 +174,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContext(0, 0, 0) + val tContext = new TaskContextImpl(0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index faba5508c906c..561a5e9cd90c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -51,7 +51,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } test("all TaskCompletionListeners should be called even if some fail") { - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val listener = mock(classOf[TaskCompletionListener]) context.addTaskCompletionListener(_ => throw new Exception("blah")) context.addTaskCompletionListener(listener) diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 809bd70929656..a8c049d749015 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.apache.spark.TaskContext +import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} import org.mockito.Mockito._ @@ -62,7 +62,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, @@ -120,7 +120,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, @@ -169,7 +169,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { (bmId, Seq((blId1, 1L), (blId2, 1L)))) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 39f8ba4745737..d919b18e09855 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -32,7 +32,7 @@ object MimaBuild { ProblemFilters.exclude[MissingMethodProblem](fullName), // Sometimes excluded methods have default arguments and // they are translated into public methods/fields($default$) in generated - // bytecode. It is not possible to exhustively list everything. + // bytecode. It is not possible to exhaustively list everything. // But this should be okay. ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$2"), ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$1"), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index d499302124461..350aad47735e4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -50,7 +50,11 @@ object MimaExcludes { "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2"), // MapStatus should be private[spark] ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.scheduler.MapStatus") + "org.apache.spark.scheduler.MapStatus"), + // TaskContext was promoted to Abstract class + ProblemFilters.exclude[AbstractClassProblem]( + "org.apache.spark.TaskContext") + ) case v if v.startsWith("1.1") => 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 1f4237d7ede65..5c6fa78ae3895 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 @@ -289,9 +289,9 @@ case class InsertIntoParquetTable( def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = new AppendingParquetOutputFormat(taskIdOffset) From 7f7b50ed9d4ffdd6b23e0faa56b068a049da67f7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 16 Oct 2014 18:58:18 -0700 Subject: [PATCH 05/32] [SPARK-3923] Increase Akka heartbeat pause above heartbeat interval Something about the 2.3.4 upgrade seems to have made the issue manifest where all the services disconnect from each other after exactly 1000 seconds (which is the heartbeat interval). [This post](https://groups.google.com/forum/#!topic/akka-user/X3xzpTCbEFs) suggests that heartbeat pause should be greater than heartbeat interval, and increasing the pause from 600s to 6000s seems to have rectified the issue. My current cluster has now exceeded 1400s of uptime without failure! I do not know why this fixed it, because the threshold we have set for the failure detector is the exponent of a timeout, and 300 is extremely large. Perhaps the default failure detector changed in 2.3.4 and now ignores threshold. Author: Aaron Davidson Closes #2784 from aarondav/fix-timeout and squashes the following commits: bd1151a [Aaron Davidson] Increase pause, don't decrease interval 9cb0372 [Aaron Davidson] [SPARK-3923] Decrease Akka heartbeat interval below heartbeat pause --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 2 +- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 e2d32c859bbda..f41c8d0315cb3 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -77,7 +77,7 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 600) + val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) val akkaFailureDetector = conf.getDouble("spark.akka.failure-detector.threshold", 300.0) val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) diff --git a/docs/configuration.md b/docs/configuration.md index f311f0d2a6206..8515ee045177f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -725,7 +725,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 600 + 6000 This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause From be2ec4a91d14f48e6323989fb0e0226a9d65bf7e Mon Sep 17 00:00:00 2001 From: Kun Li Date: Thu, 16 Oct 2014 19:00:10 -0700 Subject: [PATCH 06/32] [SQL]typo in HiveFromSpark Author: Kun Li Closes #2809 from jackylk/patch-1 and squashes the following commits: 46c926b [Kun Li] typo in HiveFromSpark --- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index e26f213e8afa8..0c52ef8ed96ac 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -28,7 +28,7 @@ object HiveFromSpark { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) - // A local hive context creates an instance of the Hive Metastore in process, storing the + // A local hive context creates an instance of the Hive Metastore in process, storing // the warehouse data in the current directory. This location can be overridden by // specifying a second parameter to the constructor. val hiveContext = new HiveContext(sc) From 642b246beb7879978d31f2e6e97de7e06c74dcb7 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 16 Oct 2014 19:07:37 -0700 Subject: [PATCH 07/32] [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfo In BlockManagermasterActor, _remainingMem would increase memSize for twice when updateBlockInfo if new storageLevel is invalid and old storageLevel is "useMemory". Also, _remainingMem should increase with original memory size instead of new memSize. Author: Zhang, Liye Closes #2792 from liyezhang556520/spark-3941-remainMem and squashes the following commits: 3d487cc [Zhang, Liye] make the code concise 0380a32 [Zhang, Liye] [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfo --- .../apache/spark/storage/BlockManagerMasterActor.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 6a06257ed0c08..088f06e389d83 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -457,16 +457,18 @@ private[spark] class BlockManagerInfo( if (_blocks.containsKey(blockId)) { // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + val blockStatus: BlockStatus = _blocks.get(blockId) + val originalLevel: StorageLevel = blockStatus.storageLevel + val originalMemSize: Long = blockStatus.memSize if (originalLevel.useMemory) { - _remainingMem += memSize + _remainingMem += originalMemSize } } if (storageLevel.isValid) { /* isValid means it is either stored in-memory, on-disk or on-Tachyon. - * But the memSize here indicates the data size in or dropped from memory, + * The memSize here indicates the data size in or dropped from memory, * tachyonSize here indicates the data size in or dropped from Tachyon, * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. @@ -493,7 +495,6 @@ private[spark] class BlockManagerInfo( val blockStatus: BlockStatus = _blocks.get(blockId) _blocks.remove(blockId) if (blockStatus.storageLevel.useMemory) { - _remainingMem += blockStatus.memSize logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), Utils.bytesToString(_remainingMem))) From e7f4ea8a52f0d3d56684b4f9caadce978eac4816 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 16 Oct 2014 19:12:39 -0700 Subject: [PATCH 08/32] [SPARK-3890][Docs]remove redundant spark.executor.memory in doc Introduced in https://github.com/pwendell/spark/commit/f7e79bc42c1635686c3af01eef147dae92de2529, I'm not sure why we need two spark.executor.memory here. Author: WangTaoTheTonic Author: WangTao Closes #2745 from WangTaoTheTonic/redundantconfig and squashes the following commits: e7564dc [WangTao] too long line fdbdb1f [WangTaoTheTonic] trivial workaround d06b6e5 [WangTaoTheTonic] remove redundant spark.executor.memory in doc --- docs/configuration.md | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8515ee045177f..f0204c640bc89 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -161,14 +161,6 @@ Apart from these, the following properties are also available, and may be useful #### Runtime Environment - - - - - @@ -365,7 +357,7 @@ Apart from these, the following properties are also available, and may be useful @@ -880,8 +872,8 @@ Apart from these, the following properties are also available, and may be useful @@ -893,7 +885,7 @@ Apart from these, the following properties are also available, and may be useful to wait for before scheduling begins. Specified as a double between 0 and 1. Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config - spark.scheduler.maxRegisteredResourcesWaitingTime + spark.scheduler.maxRegisteredResourcesWaitingTime. From 56fd34af52a18230bf3ea7b041f2a184eddc1103 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Oct 2014 19:22:02 -0700 Subject: [PATCH 09/32] [SPARK-3741] Add afterExecute for handleConnectExecutor Sorry. I found that I forgot to add `afterExecute` for `handleConnectExecutor` in #2593. Author: zsxwing Closes #2794 from zsxwing/SPARK-3741 and squashes the following commits: a0bc4dd [zsxwing] Add afterExecute for handleConnectExecutor --- .../apache/spark/network/nio/ConnectionManager.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 9396b6ba84e7e..bda4bf50932c3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -117,7 +117,16 @@ private[nio] class ConnectionManager( conf.getInt("spark.core.connection.connect.threads.max", 8), conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-connect-executor")) + Utils.namedThreadFactory("handle-connect-executor")) { + + override def afterExecute(r: Runnable, t: Throwable): Unit = { + super.afterExecute(r, t) + if (t != null && NonFatal(t)) { + logError("Error in handleConnectExecutor is not handled properly", t) + } + } + + } private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation From dedace83f35cba0f833d962acbd75572318948c4 Mon Sep 17 00:00:00 2001 From: yantangzhai Date: Thu, 16 Oct 2014 19:25:37 -0700 Subject: [PATCH 10/32] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes JobProgressPage could not show Fair Scheduler Pools section sometimes. SparkContext starts webui and then postEnvironmentUpdate. Sometimes JobProgressPage is accessed between webui starting and postEnvironmentUpdate, then the lazy val isFairScheduler will be false. The Fair Scheduler Pools section will not display any more. Author: yantangzhai Author: YanTangZhai Closes #1966 from YanTangZhai/SPARK-3067 and squashes the following commits: d4323f8 [yantangzhai] update [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes 8a00106 [YanTangZhai] Merge pull request #6 from apache/master b6391cc [yantangzhai] revert [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes d2226cd [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cbcba66 [YanTangZhai] Merge pull request #3 from apache/master aac7f7b [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cdef539 [YanTangZhai] Merge pull request #1 from apache/master --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b709b8880ba76..354116286c77d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -238,7 +238,6 @@ class SparkContext(config: SparkConf) extends Logging { // For tests, do not enable the UI None } - ui.foreach(_.bind()) /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -342,6 +341,10 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() + // Bind the SparkUI after starting the task scheduler + // because certain pages and listeners depend on it + ui.foreach(_.bind()) + private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack From e678b9f02a2936b35c95e91a5f0ff388b5720261 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 16 Oct 2014 19:43:33 -0700 Subject: [PATCH 11/32] [SPARK-3973] Print call site information for broadcasts Its hard to debug which broadcast variables refer to what in a big codebase. Printing call site information helps in debugging. Author: Shivaram Venkataraman Closes #2829 from shivaram/spark-broadcast-print and squashes the following commits: cd6dbdf [Shivaram Venkataraman] Print call site information for broadcasts --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 354116286c77d..dd3157990ef2d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -818,6 +818,8 @@ class SparkContext(config: SparkConf) extends Logging { */ def broadcast[T: ClassTag](value: T): Broadcast[T] = { val bc = env.broadcastManager.newBroadcast[T](value, isLocal) + val callSite = getCallSite + logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm) cleaner.foreach(_.registerBroadcastForCleanup(bc)) bc } From c351862064ed7d2031ea4c8bf33881e5f702ea0a Mon Sep 17 00:00:00 2001 From: likun Date: Fri, 17 Oct 2014 10:33:45 -0700 Subject: [PATCH 12/32] [SPARK-3935][Core] log the number of records that has been written There is a unused variable(count) in saveAsHadoopDataset in PairRDDFunctions.scala. The initial idea of this variable seems to count the number of records, so I am adding a log statement to log the number of records that has been written to the writer. Author: likun Author: jackylk Closes #2791 from jackylk/SPARK-3935 and squashes the following commits: a874047 [jackylk] removing the unused variable in PairRddFunctions.scala 3bf43c7 [likun] log the number of records has been written --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 929ded58a3bd5..ac96de86dd6d4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1032,10 +1032,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() try { - var count = 0 while (iter.hasNext) { val record = iter.next() - count += 1 writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) } } finally { From 803e7f087797bae643754f8db88848a17282ca6e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Oct 2014 13:45:10 -0500 Subject: [PATCH 13/32] [SPARK-3979] [yarn] Use fs's default replication. This avoids issues when HDFS is configured in a way that would not allow the hardcoded default replication of "3". Note: getDefaultReplication(Path) was added in 0.23.3, and the oldest one available on Maven Central is 0.23.7, so I chose to not add code to access that method via reflection. Author: Marcelo Vanzin Closes #2831 from vanzin/SPARK-3979 and squashes the following commits: b0e3a97 [Marcelo Vanzin] [SPARK-3979] [yarn] Use fs's default replication. --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 14a0386b78978..0efac4ea63702 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 @@ -143,7 +143,8 @@ private[spark] trait ClientBase extends Logging { val nns = getNameNodesToAccess(sparkConf) + dst obtainTokensForNamenodes(nns, hadoopConf, credentials) - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", + fs.getDefaultReplication(dst)).toShort val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) From adcb7d3350032dda69a43de724c8bdff5fef2c67 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Oct 2014 14:12:07 -0700 Subject: [PATCH 14/32] [SPARK-3855][SQL] Preserve the result attribute of python UDFs though transformations In the current implementation it was possible for the reference to change after analysis. Author: Michael Armbrust Closes #2717 from marmbrus/pythonUdfResults and squashes the following commits: da14879 [Michael Armbrust] Fix test 6343bcb [Michael Armbrust] add test 9533286 [Michael Armbrust] Correctly preserve the result attribute of python UDFs though transformations --- python/pyspark/tests.py | 6 ++++++ .../apache/spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/execution/pythonUdfs.scala | 12 ++++++++++-- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ceab57464f013..f5ccf31abb3fa 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -683,6 +683,12 @@ def test_udf(self): [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() self.assertEqual(row[0], 5) + def test_udf2(self): + self.sqlCtx.registerFunction("strlen", lambda string: len(string)) + self.sqlCtx.inferSchema(self.sc.parallelize([Row(a="test")])).registerTempTable("test") + [res] = self.sqlCtx.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() + self.assertEqual(u"4", res[0]) + def test_broadcast_in_udf(self): bar = {"a": "aa", "b": "bb", "c": "abc"} foo = self.sc.broadcast(bar) 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 4f1af7234d551..79e4ddb8c4f5d 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 @@ -295,7 +295,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case e @ EvaluatePython(udf, child) => + case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil case _ => Nil 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 0977da3e8577c..be729e5d244b0 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 @@ -105,13 +105,21 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { } } +object EvaluatePython { + def apply(udf: PythonUDF, child: LogicalPlan) = + new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) +} + /** * :: DeveloperApi :: * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. */ @DeveloperApi -case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { - val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() +case class EvaluatePython( + udf: PythonUDF, + child: LogicalPlan, + resultAttribute: AttributeReference) + extends logical.UnaryNode { def output = child.output :+ resultAttribute } From 23f6171d633d4347ca4aa8ec7cb7bd57342b21b5 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 17 Oct 2014 14:49:44 -0700 Subject: [PATCH 15/32] [SPARK-3985] [Examples] fix file path using os.path.join Author: Daoyuan Wang Closes #2834 from adrian-wang/sqlpypath and squashes the following commits: da7aa95 [Daoyuan Wang] fix file path using path.join --- examples/src/main/python/sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index eefa022f1927c..d2c5ca48c6cb8 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -48,7 +48,7 @@ # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. - path = os.environ['SPARK_HOME'] + "examples/src/main/resources/people.json" + path = os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") # Create a SchemaRDD from the file(s) pointed to by path people = sqlContext.jsonFile(path) # root From 477c6481cca94b15c9c8b43e674f220a1cda1dd1 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 17 Oct 2014 15:02:57 -0700 Subject: [PATCH 16/32] [SPARK-3934] [SPARK-3918] [mllib] Bug fixes for RandomForest, DecisionTree SPARK-3934: When run with a mix of unordered categorical and continuous features, on multiclass classification, RandomForest fails. The bug is in the sanity checks in getFeatureOffset and getLeftRightFeatureOffsets, which use the wrong indices for checking whether features are unordered. Fix: Remove the sanity checks since they are not really needed, and since they would require DTStatsAggregator to keep track of an extra set of indices (for the feature subset). Added test to RandomForestSuite which failed with old version but now works. SPARK-3918: Added baggedInput.unpersist at end of training. Also: * I removed DTStatsAggregator.isUnordered since it is no longer used. * DecisionTreeMetadata: Added logWarning when maxBins is automatically reduced. * Updated DecisionTreeRunner to explicitly fix the test data to have the same number of features as the training data. This is a temporary fix which should eventually be replaced by pre-indexing both datasets. * RandomForestModel: Updated toString to print total number of nodes in forest. * Changed Predict class to be public DeveloperApi. This was necessary to allow users to create their own trees by hand (for testing). CC: mengxr manishamde chouqin codedeft Just notifying you of these small bug fixes. Author: Joseph K. Bradley Closes #2785 from jkbradley/dtrunner-update and squashes the following commits: 9132321 [Joseph K. Bradley] merged with master, fixed imports 9dbd000 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update e116473 [Joseph K. Bradley] Changed Predict class to be public DeveloperApi. f502e65 [Joseph K. Bradley] bug fix for SPARK-3934 7f3d60f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update ba567ab [Joseph K. Bradley] Changed DTRunner to load test data using same number of features as in training data. 4e88c1f [Joseph K. Bradley] changed RF toString to print total number of nodes --- .../examples/mllib/DecisionTreeRunner.scala | 3 ++- .../mllib/tree/impl/DTStatsAggregator.scala | 16 +--------------- .../mllib/tree/impl/DecisionTreeMetadata.scala | 7 ++++++- .../apache/spark/mllib/tree/model/Predict.scala | 5 ++++- .../mllib/tree/model/RandomForestModel.scala | 4 ++-- .../spark/mllib/tree/RandomForestSuite.scala | 16 ++++++++++++++++ 6 files changed, 31 insertions(+), 20 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 837d0591478c5..0890e6263e165 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -189,9 +189,10 @@ object DecisionTreeRunner { // Create training, test sets. val splits = if (params.testInput != "") { // Load testInput. + val numFeatures = examples.take(1)(0).features.size val origTestExamples = params.dataFormat match { case "dense" => MLUtils.loadLabeledPoints(sc, params.testInput) - case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput) + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput, numFeatures) } params.algo match { case Classification => { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 55f422dff0d71..ce8825cc03229 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -64,12 +64,6 @@ private[tree] class DTStatsAggregator( numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) } - /** - * Indicator for each feature of whether that feature is an unordered feature. - * TODO: Is Array[Boolean] any faster? - */ - def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) - /** * Total number of elements stored in this aggregator */ @@ -128,21 +122,13 @@ private[tree] class DTStatsAggregator( * Pre-compute feature offset for use with [[featureUpdate]]. * For ordered features only. */ - def getFeatureOffset(featureIndex: Int): Int = { - require(!isUnordered(featureIndex), - s"DTStatsAggregator.getFeatureOffset is for ordered features only, but was called" + - s" for unordered feature $featureIndex.") - featureOffsets(featureIndex) - } + def getFeatureOffset(featureIndex: Int): Int = featureOffsets(featureIndex) /** * Pre-compute feature offset for use with [[featureUpdate]]. * For unordered features only. */ def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { - require(isUnordered(featureIndex), - s"DTStatsAggregator.getLeftRightFeatureOffsets is for unordered features only," + - s" but was called for ordered feature $featureIndex.") val baseOffset = featureOffsets(featureIndex) (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } 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 index 212dce25236e0..772c02670e541 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.impl import scala.collection.mutable +import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ @@ -82,7 +83,7 @@ private[tree] class DecisionTreeMetadata( } -private[tree] object DecisionTreeMetadata { +private[tree] object DecisionTreeMetadata extends Logging { /** * Construct a [[DecisionTreeMetadata]] instance for this dataset and parameters. @@ -103,6 +104,10 @@ private[tree] object DecisionTreeMetadata { } val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + if (maxPossibleBins < strategy.maxBins) { + logWarning(s"DecisionTree reducing maxBins from ${strategy.maxBins} to $maxPossibleBins" + + s" (= number of training instances)") + } // We check the number of bins here against maxPossibleBins. // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index d8476b5cd7bc7..004838ee5ba0e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,12 +17,15 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi + /** * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) */ -private[tree] class Predict( +@DeveloperApi +class Predict( val predict: Double, val prob: Double = 0.0) extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala index 4d66d6d81caa5..6a22e2abe59bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala @@ -82,9 +82,9 @@ class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) ext */ override def toString: String = algo match { case Classification => - s"RandomForestModel classifier with $numTrees trees" + s"RandomForestModel classifier with $numTrees trees and $totalNumNodes total nodes" case Regression => - s"RandomForestModel regressor with $numTrees trees" + s"RandomForestModel regressor with $numTrees trees and $totalNumNodes total nodes" case _ => throw new IllegalArgumentException( s"RandomForestModel given unknown algo parameter: $algo.") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 20d372dc1d3ca..fb44ceb0f57ee 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -173,6 +173,22 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) } + test("alternating categorical and continuous features with multiclass labels to test indexing") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0, 3.0, 1.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0, 1.0, 2.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0, 6.0, 3.0)) + arr(3) = new LabeledPoint(2.0, Vectors.dense(0.0, 2.0, 1.0, 3.0, 2.0)) + val categoricalFeaturesInfo = Map(0 -> 3, 2 -> 2, 4 -> 4) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) + val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, + featureSubsetStrategy = "sqrt", seed = 12345) + RandomForestSuite.validateClassifier(model, arr, 1.0) + } + } object RandomForestSuite { From f406a8391825d8866110f29a0d656c82cd064520 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 18 Oct 2014 12:33:20 -0700 Subject: [PATCH 17/32] SPARK-3926 [CORE] Result of JavaRDD.collectAsMap() is not Serializable Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are Author: Sean Owen Closes #2805 from srowen/SPARK-3926 and squashes the following commits: ecb78ee [Sean Owen] Fix conflict between java.io.Serializable and use of Scala's Serializable f4717f9 [Sean Owen] Oops, fix compile problem ae1b36f [Sean Owen] Expand to cover Maps returned from other Java API methods as well 51c26c2 [Sean Owen] Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are --- .../org/apache/spark/api/java/JavaPairRDD.scala | 12 +++++++----- .../org/apache/spark/api/java/JavaRDDLike.scala | 7 ++++--- .../scala/org/apache/spark/api/java/JavaUtils.scala | 10 ++++++++++ .../scala/org/apache/spark/sql/api/java/Row.scala | 3 ++- 4 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 0846225e4f992..c38b96528d037 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} @@ -265,10 +266,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * before sending results to a reducer, similarly to a "combiner" in MapReduce. */ def reduceByKeyLocally(func: JFunction2[V, V, V]): java.util.Map[K, V] = - mapAsJavaMap(rdd.reduceByKeyLocally(func)) + mapAsSerializableJavaMap(rdd.reduceByKeyLocally(func)) /** Count the number of elements for each key, and return the result to the master as a Map. */ - def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) + def countByKey(): java.util.Map[K, Long] = mapAsSerializableJavaMap(rdd.countByKey()) /** * :: Experimental :: @@ -277,7 +278,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = - rdd.countByKeyApprox(timeout).map(mapAsJavaMap) + rdd.countByKeyApprox(timeout).map(mapAsSerializableJavaMap) /** * :: Experimental :: @@ -287,7 +288,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = - rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + rdd.countByKeyApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** * Aggregate the values of each key, using given combine functions and a neutral "zero value". @@ -614,7 +615,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return the key-value pairs in this RDD to the master as a Map. */ - def collectAsMap(): java.util.Map[K, V] = mapAsJavaMap(rdd.collectAsMap()) + def collectAsMap(): java.util.Map[K, V] = mapAsSerializableJavaMap(rdd.collectAsMap()) + /** * Pass each value in the key-value pair RDD through a map function without changing the keys; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 545bc0e9e99ed..c744399483349 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -30,6 +30,7 @@ import org.apache.spark.{FutureAction, Partition, SparkContext, TaskContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -390,7 +391,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): java.util.Map[T, java.lang.Long] = - mapAsJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) + mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) /** * (Experimental) Approximate version of countByValue(). @@ -399,13 +400,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { timeout: Long, confidence: Double ): PartialResult[java.util.Map[T, BoundedDouble]] = - rdd.countByValueApprox(timeout, confidence).map(mapAsJavaMap) + rdd.countByValueApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** * (Experimental) Approximate version of countByValue(). */ def countByValueApprox(timeout: Long): PartialResult[java.util.Map[T, BoundedDouble]] = - rdd.countByValueApprox(timeout).map(mapAsJavaMap) + rdd.countByValueApprox(timeout).map(mapAsSerializableJavaMap) /** * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 22810cb1c662d..b52d0a5028e84 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,10 +19,20 @@ package org.apache.spark.api.java import com.google.common.base.Optional +import scala.collection.convert.Wrappers.MapWrapper + private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) case None => Optional.absent() } + + // Workaround for SPARK-3926 / SI-8911 + def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = + new SerializableMapWrapper(underlying) + + class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) + extends MapWrapper(underlying) with java.io.Serializable + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index e9d04ce7aae4c..df01411f60a05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -22,6 +22,7 @@ import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} import scala.collection.JavaConversions import scala.math.BigDecimal +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** @@ -114,7 +115,7 @@ object Row { // they are actually accessed. case row: ScalaRow => new Row(row) case map: scala.collection.Map[_, _] => - JavaConversions.mapAsJavaMap( + mapAsSerializableJavaMap( map.map { case (key, value) => (toJavaValue(key), toJavaValue(value)) } From 05db2da7dc256822cdb602c4821cbb9fb84dac98 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 18 Oct 2014 19:14:48 -0700 Subject: [PATCH 18/32] [SPARK-3952] [Streaming] [PySpark] add Python examples in Streaming Programming Guide Having Python examples in Streaming Programming Guide. Also add RecoverableNetworkWordCount example. Author: Davies Liu Author: Davies Liu Closes #2808 from davies/pyguide and squashes the following commits: 8d4bec4 [Davies Liu] update readme 26a7e37 [Davies Liu] fix format 3821c4d [Davies Liu] address comments, add missing file 7e4bb8a [Davies Liu] add Python examples in Streaming Programming Guide --- docs/README.md | 3 +- docs/streaming-programming-guide.md | 304 +++++++++++++++++- .../recoverable_network_wordcount.py | 80 +++++ python/docs/pyspark.streaming.rst | 10 + python/pyspark/streaming/dstream.py | 8 +- 5 files changed, 391 insertions(+), 14 deletions(-) create mode 100644 examples/src/main/python/streaming/recoverable_network_wordcount.py create mode 100644 python/docs/pyspark.streaming.rst diff --git a/docs/README.md b/docs/README.md index 0facecdd5f767..d2d58e435d4c4 100644 --- a/docs/README.md +++ b/docs/README.md @@ -25,8 +25,7 @@ installing via the Ruby Gem dependency manager. Since the exact HTML output varies between versions of Jekyll and its dependencies, we list specific versions here in some cases: - $ sudo gem install jekyll -v 1.4.3 - $ sudo gem uninstall kramdown -v 1.4.1 + $ sudo gem install jekyll $ sudo gem install jekyll-redirect-from Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 738309c668387..8bbba88b31978 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -212,6 +212,67 @@ The complete code can be found in the Spark Streaming example [JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
+ +
+First, we import StreamingContext, which is the main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +# Create a local StreamingContext with two working thread and batch interval of 1 second +sc = SparkContext("local[2]", "NetworkWordCount") +ssc = StreamingContext(sc, 1) +{% endhighlight %} + +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` + +{% highlight python %} +# Create a DStream that will connect to hostname:port, like localhost:9999 +lines = ssc.socketTextStream("localhost", 9999) +{% endhighlight %} + +This `lines` DStream represents the stream of data that will be received from the data +server. Each record in this DStream is a line of text. Next, we want to split the lines by +space into words. + +{% highlight python %} +# Split each line into words +words = lines.flatMap(lambda line: line.split(" ")) +{% endhighlight %} + +`flatMap` is a one-to-many DStream operation that creates a new DStream by +generating multiple new records from each record in the source DStream. In this case, +each line will be split into multiple words and the stream of words is represented as the +`words` DStream. Next, we want to count these words. + +{% highlight python %} +# Count each word in each batch +pairs = words.map(lambda word: (word, 1)) +wordCounts = pairs.reduceByKey(lambda x, y: x + y) + +# Print the first ten elements of each RDD generated in this DStream to the console +wordCounts.pprint() +{% endhighlight %} + +The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, +1)` pairs, which is then reduced to get the frequency of words in each batch of data. +Finally, `wordCounts.pprint()` will print a few of the counts generated every second. + +Note that when these lines are executed, Spark Streaming only sets up the computation it +will perform when it is started, and no real processing has started yet. To start the processing +after all the transformations have been setup, we finally call + +{% highlight python %} +ssc.start() # Start the computation +ssc.awaitTermination() # Wait for the computation to terminate +{% endhighlight %} + +The complete code can be found in the Spark Streaming example +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/network_wordcount.py). +
+
@@ -236,6 +297,11 @@ $ ./bin/run-example streaming.NetworkWordCount localhost 9999 $ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %} +
+{% highlight bash %} +$ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 +{% endhighlight %} +
@@ -259,8 +325,11 @@ hello world
Property NameDefaultMeaning
spark.executor.memory512m - Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. 512m, 2g). -
spark.executor.extraJavaOptions (none)spark.ui.port 4040 - Port for your application's dashboard, which shows memory and workload data + Port for your application's dashboard, which shows memory and workload data.
spark.scheduler.revive.interval 1000 - The interval length for the scheduler to revive the worker resource offers to run tasks. - (in milliseconds) + The interval length for the scheduler to revive the worker resource offers to run tasks + (in milliseconds).
+
+ +
{% highlight bash %} -# TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount +# TERMINAL 2: RUNNING NetworkWordCount $ ./bin/run-example streaming.NetworkWordCount localhost 9999 ... @@ -271,6 +340,37 @@ Time: 1357008430000 ms (world,1) ... {% endhighlight %} +
+ +
+{% highlight bash %} +# TERMINAL 2: RUNNING JavaNetworkWordCount + +$ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 +... +------------------------------------------- +Time: 1357008430000 ms +------------------------------------------- +(hello,1) +(world,1) +... +{% endhighlight %} +
+
+{% highlight bash %} +# TERMINAL 2: RUNNING network_wordcount.py + +$ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 +... +------------------------------------------- +Time: 2014-10-14 15:25:21 +------------------------------------------- +(hello,1) +(world,1) +... +{% endhighlight %} +
+
@@ -398,9 +498,34 @@ JavaSparkContext sc = ... //existing JavaSparkContext JavaStreamingContext ssc = new JavaStreamingContext(sc, new Duration(1000)); {% endhighlight %} +

+ +A [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) object can be created from a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +sc = SparkContext(master, appName) +ssc = StreamingContext(sc, 1) +{% endhighlight %} + +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[\*]" to run Spark Streaming +in-process (detects the number of cores in the local system). + +The batch interval must be set based on the latency requirements of your application +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +section for more details. +
After a context is defined, you have to do the follow steps. + 1. Define the input sources. 1. Setup the streaming computations. 1. Start the receiving and procesing of data using `streamingContext.start()`. @@ -483,6 +608,9 @@ methods for creating DStreams from files and Akka actors as input sources.
streamingContext.fileStream(dataDirectory);
+
+ streamingContext.textFileStream(dataDirectory) +
Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory (files written in nested directories not supported). Note that @@ -684,13 +812,30 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi JavaPairDStream runningCounts = pairs.updateStateByKey(updateFunction); {% endhighlight %} + +
+ +{% highlight python %} +def updateFunction(newValues, runningCount): + if runningCount is None: + runningCount = 0 + return sum(newValues, runningCount) # add the new values with the previous running count to get the new count +{% endhighlight %} + +This is applied on a DStream containing words (say, the `pairs` DStream containing `(word, +1)` pairs in the [earlier example](#a-quick-example)). + +{% highlight python %} +runningCounts = pairs.updateStateByKey(updateFunction) +{% endhighlight %} +
The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Scala code, take a look at the example -[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala). +[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py). #### Transform Operation {:.no_toc} @@ -732,6 +877,15 @@ JavaPairDStream cleanedDStream = wordCounts.transform( }); {% endhighlight %} + +
+ +{% highlight python %} +spamInfoRDD = sc.pickleFile(...) # RDD containing spam information + +# join data stream with spam information to do data cleaning +cleanedDStream = wordCounts.transform(lambda rdd: rdd.join(spamInfoRDD).filter(...)) +{% endhighlight %}
@@ -793,6 +947,14 @@ Function2 reduceFunc = new Function2 windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); {% endhighlight %} + +
+ +{% highlight python %} +# Reduce last 30 seconds of data, every 10 seconds +windowedWordCounts = pairs.reduceByKeyAndWindow(lambda x, y: x + y, lambda x, y: x - y, 30, 10) +{% endhighlight %} +
@@ -860,6 +1022,7 @@ see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). +For the Python API, see [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) *** @@ -872,9 +1035,12 @@ Currently, the following output operations are defined: - + + This is useful for development and debugging. +
+ PS: called pprint() in Python) + @@ -915,17 +1081,41 @@ For this purpose, a developer may inadvertantly try creating a connection object the Spark driver, but try to use it in a Spark worker to save records in the RDDs. For example (in Scala), +
+
+ +{% highlight scala %} dstream.foreachRDD(rdd => { val connection = createNewConnection() // executed at the driver rdd.foreach(record => { connection.send(record) // executed at the worker }) }) +{% endhighlight %} + +
+
+ +{% highlight python %} +def sendRecord(rdd): + connection = createNewConnection() # executed at the driver + rdd.foreach(lambda record: connection.send(record)) + connection.close() + +dstream.foreachRDD(sendRecord) +{% endhighlight %} + +
+
- This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. + This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. - However, this can lead to another common mistake - creating a new connection for every record. For example, +
+
+ +{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreach(record => { val connection = createNewConnection() @@ -933,9 +1123,28 @@ For example (in Scala), connection.close() }) }) +{% endhighlight %} - Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. +
+
+ +{% highlight python %} +def sendRecord(record): + connection = createNewConnection() + connection.send(record) + connection.close() + +dstream.foreachRDD(lambda rdd: rdd.foreach(sendRecord)) +{% endhighlight %} +
+
+ + Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. + +
+
+{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreachPartition(partitionOfRecords => { val connection = createNewConnection() @@ -943,13 +1152,31 @@ For example (in Scala), connection.close() }) }) +{% endhighlight %} +
+ +
+{% highlight python %} +def sendPartition(iter): + connection = createNewConnection() + for record in iter: + connection.send(record) + connection.close() + +dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) +{% endhighlight %} +
+
- This amortizes the connection creation overheads over many records. + This amortizes the connection creation overheads over many records. - Finally, this can be further optimized by reusing connection objects across multiple RDDs/batches. One can maintain a static pool of connection objects than can be reused as RDDs of multiple batches are pushed to the external system, thus further reducing the overheads. - + +
+
+{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreachPartition(partitionOfRecords => { // ConnectionPool is a static, lazily initialized pool of connections @@ -958,8 +1185,25 @@ For example (in Scala), ConnectionPool.returnConnection(connection) // return to the pool for future reuse }) }) +{% endhighlight %} +
- Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. +
+{% highlight python %} +def sendPartition(iter): + # ConnectionPool is a static, lazily initialized pool of connections + connection = ConnectionPool.getConnection() + for record in iter: + connection.send(record) + # return to the pool for future reuse + ConnectionPool.returnConnection(connection) + +dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) +{% endhighlight %} +
+
+ +Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. ##### Other points to remember: @@ -1376,6 +1620,44 @@ You can also explicitly create a `JavaStreamingContext` from the checkpoint data the computation by using `new JavaStreamingContext(checkpointDirectory)`. +
+ +This behavior is made simple by using `StreamingContext.getOrCreate`. This is used as follows. + +{% highlight python %} +# Function to create and setup a new StreamingContext +def functionToCreateContext(): + sc = SparkContext(...) # new context + ssc = new StreamingContext(...) + lines = ssc.socketTextStream(...) # create DStreams + ... + ssc.checkpoint(checkpointDirectory) # set checkpoint directory + return ssc + +# Get StreamingContext from checkpoint data or create a new one +context = StreamingContext.getOrCreate(checkpointDirectory, functionToCreateContext) + +# Do additional setup on context that needs to be done, +# irrespective of whether it is being started or restarted +context. ... + +# Start the context +context.start() +context.awaitTermination() +{% endhighlight %} + +If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data. +If the directory does not exist (i.e., running for the first time), +then the function `functionToCreateContext` will be called to create a new +context and set up the DStreams. See the Python example +[recoverable_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/streaming/recoverable_network_wordcount.py). +This example appends the word counts of network data into a file. + +You can also explicitly create a `StreamingContext` from the checkpoint data and start the + computation by using `StreamingContext.getOrCreate(checkpointDirectory, None)`. + +
+ **Note**: If Spark Streaming and/or the Spark Streaming program is recompiled, @@ -1572,7 +1854,11 @@ package and renamed for better clarity. [TwitterUtils](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html), [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) + - Python docs + * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) + * [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) + and [Python] ({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/streaming) * [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and [video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py new file mode 100644 index 0000000000000..fc6827c82bf9b --- /dev/null +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -0,0 +1,80 @@ +# +# 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. +# + +""" + Counts words in text encoded with UTF8 received from the network every second. + + Usage: recoverable_network_wordcount.py + and describe the TCP server that Spark Streaming would connect to receive + data. directory to HDFS-compatible file system which checkpoint data + file to which the word counts will be appended + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/recoverable_network_wordcount.py \ + localhost 9999 ~/checkpoint/ ~/out` + + If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create + a new StreamingContext (will print "Creating new context" to the console). Otherwise, if + checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from + the checkpoint data. +""" + +import os +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + + +def createContext(host, port, outputPath): + # If you do not see this printed, that means the StreamingContext has been loaded + # from the new checkpoint + print "Creating new context" + if os.path.exists(outputPath): + os.remove(outputPath) + sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") + ssc = StreamingContext(sc, 1) + + # Create a socket stream on target ip:port and count the + # words in input stream of \n delimited text (eg. generated by 'nc') + lines = ssc.socketTextStream(host, port) + words = lines.flatMap(lambda line: line.split(" ")) + wordCounts = words.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y) + + def echo(time, rdd): + counts = "Counts at time %s %s" % (time, rdd.collect()) + print counts + print "Appending to " + os.path.abspath(outputPath) + with open(outputPath, 'a') as f: + f.write(counts + "\n") + + wordCounts.foreachRDD(echo) + return ssc + +if __name__ == "__main__": + if len(sys.argv) != 5: + print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ + " " + exit(-1) + host, port, checkpoint, output = sys.argv[1:] + ssc = StreamingContext.getOrCreate(checkpoint, + lambda: createContext(host, int(port), output)) + ssc.start() + ssc.awaitTermination() diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst new file mode 100644 index 0000000000000..5024d694b668f --- /dev/null +++ b/python/docs/pyspark.streaming.rst @@ -0,0 +1,10 @@ +pyspark.streaming module +================== + +Module contents +--------------- + +.. automodule:: pyspark.streaming + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 5ae5cf07f0137..0826ddc56e844 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -441,9 +441,11 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio if `invReduceFunc` is not None, the reduction is done incrementally using the old window's reduced value : - 1. reduce the new values that entered the window (e.g., adding new counts) - 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - This is more efficient than `invReduceFunc` is None. + + 1. reduce the new values that entered the window (e.g., adding new counts) + + 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + This is more efficient than `invReduceFunc` is None. @param reduceFunc: associative reduce function @param invReduceFunc: inverse reduce function of `reduceFunc` From 7e63bb49c526c3f872619ae14e4b5273f4c535e9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Oct 2014 00:31:06 -0700 Subject: [PATCH 19/32] [SPARK-2546] Clone JobConf for each task (branch-1.0 / 1.1 backport) This patch attempts to fix SPARK-2546 in `branch-1.0` and `branch-1.1`. The underlying problem is that thread-safety issues in Hadoop Configuration objects may cause Spark tasks to get stuck in infinite loops. The approach taken here is to clone a new copy of the JobConf for each task rather than sharing a single copy between tasks. Note that there are still Configuration thread-safety issues that may affect the driver, but these seem much less likely to occur in practice and will be more complex to fix (see discussion on the SPARK-2546 ticket). This cloning is guarded by a new configuration option (`spark.hadoop.cloneConf`) and is disabled by default in order to avoid unexpected performance regressions for workloads that are unaffected by the Configuration thread-safety issues. Author: Josh Rosen Closes #2684 from JoshRosen/jobconf-fix-backport and squashes the following commits: f14f259 [Josh Rosen] Add configuration option to control cloning of Hadoop JobConf. b562451 [Josh Rosen] Remove unused jobConfCacheKey field. dd25697 [Josh Rosen] [SPARK-2546] [1.0 / 1.1 backport] Clone JobConf for each task. (cherry picked from commit 2cd40db2b3ab5ddcb323fd05c171dbd9025f9e71) Signed-off-by: Josh Rosen Conflicts: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala --- .../org/apache/spark/rdd/HadoopRDD.scala | 53 +++++++++++++------ docs/configuration.md | 9 ++++ 2 files changed, 47 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 8010dd90082f8..775141775e06c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -132,27 +132,47 @@ class HadoopRDD[K, V]( // used to build JobTracker ID private val createTime = new Date() + private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean + // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value - if (conf.isInstanceOf[JobConf]) { - // A user-broadcasted JobConf was provided to the HadoopRDD, so always use it. - conf.asInstanceOf[JobConf] - } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) { - // getJobConf() has been called previously, so there is already a local cache of the JobConf - // needed by this RDD. - HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf] - } else { - // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the - // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). - // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. - // Synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456). + if (shouldCloneJobConf) { + // Hadoop Configuration objects are not thread-safe, which may lead to various problems if + // one job modifies a configuration while another reads it (SPARK-2546). This problem occurs + // somewhat rarely because most jobs treat the configuration as though it's immutable. One + // solution, implemented here, is to clone the Configuration object. Unfortunately, this + // clone can be very expensive. To avoid unexpected performance regressions for workloads and + // Hadoop versions that do not suffer from these thread-safety issues, this cloning is + // disabled by default. HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { + logDebug("Cloning Hadoop Configuration") val newJobConf = new JobConf(conf) - initLocalJobConfFuncOpt.map(f => f(newJobConf)) - HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + if (!conf.isInstanceOf[JobConf]) { + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + } newJobConf } + } else { + if (conf.isInstanceOf[JobConf]) { + logDebug("Re-using user-broadcasted JobConf") + conf.asInstanceOf[JobConf] + } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) { + logDebug("Re-using cached JobConf") + HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf] + } else { + // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the + // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). + // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. + // Synchronize to prevent ConcurrentModificationException (SPARK-1097, HADOOP-10456). + HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { + logDebug("Creating new JobConf and caching it for later re-use") + val newJobConf = new JobConf(conf) + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + newJobConf + } + } } } @@ -276,7 +296,10 @@ class HadoopRDD[K, V]( } private[spark] object HadoopRDD extends Logging { - /** Constructing Configuration objects is not threadsafe, use this lock to serialize. */ + /** + * Configuration's constructor is not threadsafe (see SPARK-1097 and HADOOP-10456). + * Therefore, we synchronize on this lock before calling new JobConf() or new Configuration(). + */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() /** diff --git a/docs/configuration.md b/docs/configuration.md index f0204c640bc89..96fa1377ec399 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -619,6 +619,15 @@ Apart from these, the following properties are also available, and may be useful output directories. We recommend that users do not disable this except if trying to achieve compatibility with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + + + + + From d1966f3a8bafdcef87d10ef9db5976cf89faee4b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Oct 2014 20:02:31 -0700 Subject: [PATCH 20/32] [SPARK-3902] [SPARK-3590] Stabilize AsynRDDActions and add Java API This PR adds a Java API for AsyncRDDActions and promotes the API from `Experimental` to stable. Author: Josh Rosen Author: Josh Rosen Closes #2760 from JoshRosen/async-rdd-actions-in-java and squashes the following commits: 0d45fbc [Josh Rosen] Whitespace fix. ad3ae53 [Josh Rosen] Merge remote-tracking branch 'origin/master' into async-rdd-actions-in-java c0153a5 [Josh Rosen] Remove unused variable. e8e2867 [Josh Rosen] Updates based on Marcelo's review feedback 7a1417f [Josh Rosen] Removed unnecessary java.util import. 6f8f6ac [Josh Rosen] Fix import ordering. ff28e49 [Josh Rosen] Add MiMa excludes and fix a scalastyle error. 346e46e [Josh Rosen] [SPARK-3902] Stabilize AsyncRDDActions; add Java API. --- .../spark/api/java/JavaFutureAction.java | 33 +++++++ .../scala/org/apache/spark/FutureAction.scala | 86 ++++++++++++++--- .../apache/spark/api/java/JavaRDDLike.scala | 53 ++++++++--- .../apache/spark/rdd/AsyncRDDActions.scala | 3 - .../java/org/apache/spark/JavaAPISuite.java | 93 ++++++++++++++++++- project/MimaExcludes.scala | 13 ++- 6 files changed, 246 insertions(+), 35 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java diff --git a/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java b/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java new file mode 100644 index 0000000000000..0ad189633e427 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java @@ -0,0 +1,33 @@ +/* + * 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.api.java; + + +import java.util.List; +import java.util.concurrent.Future; + +public interface JavaFutureAction extends Future { + + /** + * Returns the job IDs run by the underlying async operation. + * + * This returns the current snapshot of the job list. Certain operations may run multiple + * jobs, so multiple calls to this method may return different lists. + */ + List jobIds(); +} diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index e8f761eaa5799..d5c8f9d76c476 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -17,20 +17,21 @@ package org.apache.spark -import scala.concurrent._ -import scala.concurrent.duration.Duration -import scala.util.Try +import java.util.Collections +import java.util.concurrent.TimeUnit -import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaFutureAction import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} +import scala.concurrent._ +import scala.concurrent.duration.Duration +import scala.util.{Failure, Try} + /** - * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ -@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -69,6 +70,11 @@ trait FutureAction[T] extends Future[T] { */ override def isCompleted: Boolean + /** + * Returns whether the action has been cancelled. + */ + def isCancelled: Boolean + /** * The value of this Future. * @@ -96,15 +102,16 @@ trait FutureAction[T] extends Future[T] { /** - * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ -@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { + @volatile private var _cancelled: Boolean = false + override def cancel() { + _cancelled = true jobWaiter.cancel() } @@ -143,6 +150,8 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } override def isCompleted: Boolean = jobWaiter.jobFinished + + override def isCancelled: Boolean = _cancelled override def value: Option[Try[T]] = { if (jobWaiter.jobFinished) { @@ -164,12 +173,10 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ -@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. @@ -222,7 +229,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { // If the action hasn't been cancelled yet, submit the job. The check and the submitJob // command need to be in an atomic block. val job = this.synchronized { - if (!cancelled) { + if (!isCancelled) { rdd.context.submitJob(rdd, processPartition, partitions, resultHandler, resultFunc) } else { throw new SparkException("Action has been cancelled") @@ -243,10 +250,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { } } - /** - * Returns whether the promise has been cancelled. - */ - def cancelled: Boolean = _cancelled + override def isCancelled: Boolean = _cancelled @throws(classOf[InterruptedException]) @throws(classOf[scala.concurrent.TimeoutException]) @@ -271,3 +275,55 @@ class ComplexFutureAction[T] extends FutureAction[T] { def jobIds = jobs } + +private[spark] +class JavaFutureActionWrapper[S, T](futureAction: FutureAction[S], converter: S => T) + extends JavaFutureAction[T] { + + import scala.collection.JavaConverters._ + + override def isCancelled: Boolean = futureAction.isCancelled + + override def isDone: Boolean = { + // According to java.util.Future's Javadoc, this returns True if the task was completed, + // whether that completion was due to successful execution, an exception, or a cancellation. + futureAction.isCancelled || futureAction.isCompleted + } + + override def jobIds(): java.util.List[java.lang.Integer] = { + Collections.unmodifiableList(futureAction.jobIds.map(Integer.valueOf).asJava) + } + + private def getImpl(timeout: Duration): T = { + // This will throw TimeoutException on timeout: + Await.ready(futureAction, timeout) + futureAction.value.get match { + case scala.util.Success(value) => converter(value) + case Failure(exception) => + if (isCancelled) { + throw new CancellationException("Job cancelled").initCause(exception) + } else { + // java.util.Future.get() wraps exceptions in ExecutionException + throw new ExecutionException("Exception thrown by job", exception) + } + } + } + + override def get(): T = getImpl(Duration.Inf) + + override def get(timeout: Long, unit: TimeUnit): T = + getImpl(Duration.fromNanos(unit.toNanos(timeout))) + + override def cancel(mayInterruptIfRunning: Boolean): Boolean = synchronized { + if (isDone) { + // According to java.util.Future's Javadoc, this should return false if the task is completed. + false + } else { + // We're limited in terms of the semantics we can provide here; our cancellation is + // asynchronous and doesn't provide a mechanism to not cancel if the job is running. + futureAction.cancel() + true + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index c744399483349..efb8978f7ce12 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -21,12 +21,14 @@ import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.spark.{FutureAction, Partition, SparkContext, TaskContext} +import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag @@ -294,8 +296,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Applies a function f to all elements of this RDD. */ def foreach(f: VoidFunction[T]) { - val cleanF = rdd.context.clean((x: T) => f.call(x)) - rdd.foreach(cleanF) + rdd.foreach(x => f.call(x)) } /** @@ -576,16 +577,44 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def name(): String = rdd.name /** - * :: Experimental :: - * The asynchronous version of the foreach action. - * - * @param f the function to apply to all the elements of the RDD - * @return a FutureAction for the action + * The asynchronous version of `count`, which returns a + * future for counting the number of elements in this RDD. */ - @Experimental - def foreachAsync(f: VoidFunction[T]): FutureAction[Unit] = { - import org.apache.spark.SparkContext._ - rdd.foreachAsync(x => f.call(x)) + def countAsync(): JavaFutureAction[JLong] = { + new JavaFutureActionWrapper[Long, JLong](rdd.countAsync(), JLong.valueOf) + } + + /** + * The asynchronous version of `collect`, which returns a future for + * retrieving an array containing all of the elements in this RDD. + */ + def collectAsync(): JavaFutureAction[JList[T]] = { + new JavaFutureActionWrapper(rdd.collectAsync(), (x: Seq[T]) => x.asJava) + } + + /** + * The asynchronous version of the `take` action, which returns a + * future for retrieving the first `num` elements of this RDD. + */ + def takeAsync(num: Int): JavaFutureAction[JList[T]] = { + new JavaFutureActionWrapper(rdd.takeAsync(num), (x: Seq[T]) => x.asJava) } + /** + * The asynchronous version of the `foreach` action, which + * applies a function f to all the elements of this RDD. + */ + def foreachAsync(f: VoidFunction[T]): JavaFutureAction[Void] = { + new JavaFutureActionWrapper[Unit, Void](rdd.foreachAsync(x => f.call(x)), + { x => null.asInstanceOf[Void] }) + } + + /** + * The asynchronous version of the `foreachPartition` action, which + * applies a function f to each partition of this RDD. + */ + def foreachPartitionAsync(f: VoidFunction[java.util.Iterator[T]]): JavaFutureAction[Void] = { + new JavaFutureActionWrapper[Unit, Void](rdd.foreachPartitionAsync(x => f.call(x)), + { x => null.asInstanceOf[Void] }) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index ede5568493cc0..9f9f10b7ebc3a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,14 +24,11 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} -import org.apache.spark.annotation.Experimental /** - * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b8fa822ae4bd8..3190148fb5f43 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -20,6 +20,7 @@ import java.io.*; import java.net.URI; import java.util.*; +import java.util.concurrent.*; import scala.Tuple2; import scala.Tuple3; @@ -29,6 +30,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.base.Throwables; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -43,10 +45,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaDoubleRDD; -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.*; import org.apache.spark.api.java.function.*; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.partial.BoundedDouble; @@ -1308,6 +1307,92 @@ public void collectUnderlyingScalaRDD() { Assert.assertEquals(data.size(), collected.length); } + private static final class BuggyMapFunction implements Function { + + @Override + public T call(T x) throws Exception { + throw new IllegalStateException("Custom exception!"); + } + } + + @Test + public void collectAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.collectAsync(); + List result = future.get(); + Assert.assertEquals(data, result); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void foreachAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync( + new VoidFunction() { + @Override + public void call(Integer integer) throws Exception { + // intentionally left blank. + } + } + ); + future.get(); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void countAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.countAsync(); + long count = future.get(); + Assert.assertEquals(data.size(), count); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void testAsyncActionCancellation() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync(new VoidFunction() { + @Override + public void call(Integer integer) throws Exception { + Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. + } + }); + future.cancel(true); + Assert.assertTrue(future.isCancelled()); + Assert.assertTrue(future.isDone()); + try { + future.get(2000, TimeUnit.MILLISECONDS); + Assert.fail("Expected future.get() for cancelled job to throw CancellationException"); + } catch (CancellationException ignored) { + // pass + } + } + + @Test + public void testAsyncActionErrorWrapping() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.map(new BuggyMapFunction()).countAsync(); + try { + future.get(2, TimeUnit.SECONDS); + Assert.fail("Expected future.get() for failed job to throw ExcecutionException"); + } catch (ExecutionException ee) { + Assert.assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); + } + Assert.assertTrue(future.isDone()); + } + + /** * Test for SPARK-3647. This test needs to use the maven-built assembly to trigger the issue, * since that's the only artifact where Guava classes have been relocated. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 350aad47735e4..c58666af84f24 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,7 +54,18 @@ object MimaExcludes { // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( "org.apache.spark.TaskContext") - + ) ++ Seq( + // Adding new methods to the JavaRDDLike trait: + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.takeAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachPartitionAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.countAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.collectAsync") ) case v if v.startsWith("1.1") => From c7aeecd08fd329085760fa89025ec0d9c04f5e3f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 20 Oct 2014 10:20:21 -0700 Subject: [PATCH 21/32] [SPARK-3948][Shuffle]Fix stream corruption bug in sort-based shuffle Kernel 2.6.32 bug will lead to unexpected behavior of transferTo in copyStream, and this will corrupt the shuffle output file in sort-based shuffle, which will somehow introduce PARSING_ERROR(2), deserialization error or offset out of range. Here fix this by adding append flag, also add some position checking code. Details can be seen in [SPARK-3948](https://issues.apache.org/jira/browse/SPARK-3948). Author: jerryshao Closes #2824 from jerryshao/SPARK-3948 and squashes the following commits: be0533a [jerryshao] Address the comments a82b184 [jerryshao] add configuration to control the NIO way of copying stream e17ada2 [jerryshao] Fix kernel 2.6.32 bug led unexpected behavior of transferTo --- .../scala/org/apache/spark/util/Utils.scala | 29 ++++++++++++++++--- .../util/collection/ExternalSorter.scala | 5 ++-- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 53a7512edd852..0aeff6455b3fe 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -269,23 +269,44 @@ private[spark] object Utils extends Logging { dir } - /** Copy all data from an InputStream to an OutputStream */ + /** Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream + * copying is disabled by default unless explicitly set transferToEnabled as true, + * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. + */ def copyStream(in: InputStream, out: OutputStream, - closeStreams: Boolean = false): Long = + closeStreams: Boolean = false, + transferToEnabled: Boolean = false): Long = { var count = 0L try { - if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream]) { + if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] + && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. val inChannel = in.asInstanceOf[FileInputStream].getChannel() val outChannel = out.asInstanceOf[FileOutputStream].getChannel() + val initialPos = outChannel.position() val size = inChannel.size() // In case transferTo method transferred less data than we have required. while (count < size) { count += inChannel.transferTo(count, size - count, outChannel) } + + // Check the position after transferTo loop to see if it is in the right position and + // give user information if not. + // Position will not be increased to the expected length after calling transferTo in + // kernel version 2.6.32, this issue can be seen in + // https://bugs.openjdk.java.net/browse/JDK-7052359 + // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). + val finalPos = outChannel.position() + assert(finalPos == initialPos + size, + s""" + |Current position $finalPos do not equal to expected position ${initialPos + size} + |after transferTo, please check your kernel version to see if it is 2.6.32, + |this is a kernel bug which will lead to unexpected behavior when using transferTo. + |You can set spark.file.transferTo = false to disable this NIO feature. + """.stripMargin) } else { val buf = new Array[Byte](8192) var n = 0 @@ -727,7 +748,7 @@ private[spark] object Utils extends Logging { /** * Determines if a directory contains any files newer than cutoff seconds. - * + * * @param dir must be the path to a directory, or IllegalArgumentException is thrown * @param cutoff measured in seconds. Returns true if there are any files or directories in the * given directory whose last modified time is later than this many seconds ago diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 644fa36818647..d1b06d14acbd2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -93,6 +93,7 @@ private[spark] class ExternalSorter[K, V, C]( private val conf = SparkEnv.get.conf private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true) private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true) // Size of object batches when reading/writing from serializers. // @@ -705,10 +706,10 @@ private[spark] class ExternalSorter[K, V, C]( var out: FileOutputStream = null var in: FileInputStream = null try { - out = new FileOutputStream(outputFile) + out = new FileOutputStream(outputFile, true) for (i <- 0 until numPartitions) { in = new FileInputStream(partitionWriters(i).fileSegment().file) - val size = org.apache.spark.util.Utils.copyStream(in, out, false) + val size = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) in.close() in = null lengths(i) = size From 51afde9d8b8a67958c4632a13af143d7c7fd1f04 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Mon, 20 Oct 2014 11:01:26 -0700 Subject: [PATCH 22/32] [SPARK-4010][Web UI]Spark UI returns 500 in yarn-client mode The problem caused by #1966 CC YanTangZhai andrewor14 Author: GuoQiang Li Closes #2858 from witgo/SPARK-4010 and squashes the following commits: 9866fbf [GuoQiang Li] Spark UI returns 500 in yarn-client mode --- core/src/main/scala/org/apache/spark/SparkContext.scala | 8 ++++---- .../scala/org/apache/spark/ui/jobs/JobProgressPage.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dd3157990ef2d..ac7935b8c231e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -239,6 +239,10 @@ class SparkContext(config: SparkConf) extends Logging { None } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + ui.foreach(_.bind()) + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -341,10 +345,6 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() - // Bind the SparkUI after starting the task scheduler - // because certain pages and listeners depend on it - ui.foreach(_.bind()) - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack 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 a82f71ed08475..1e02f1225d344 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 @@ -29,7 +29,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") private val live = parent.live private val sc = parent.sc private val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { From ea054e1fc70e09e0babcdae2a37f6f7aa6a035f2 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 11:31:51 -0700 Subject: [PATCH 23/32] [SPARK-3986][SQL] Fix package names to fit their directory names. Package names of 2 test suites are different from their directory names. - `GeneratedEvaluationSuite` - `GeneratedMutableEvaluationSuite` Author: Takuya UESHIN Closes #2835 from ueshin/issues/SPARK-3986 and squashes the following commits: fa2cc05 [Takuya UESHIN] Fix package names to fit their directory names. --- .../sql/catalyst/expressions/GeneratedEvaluationSuite.scala | 3 +-- .../catalyst/expressions/GeneratedMutableEvaluationSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala index 245a2e148030c..ef3114fd4dbab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.optimizer +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 887aabb1d5fb4..275ea2627ebcd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.optimizer +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ From 4afe9a4852ebeb4cc77322a14225cd3dec165f3f Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 20 Oct 2014 11:35:18 -0700 Subject: [PATCH 24/32] [SPARK-3736] Workers reconnect when disassociated from the master. Before, if the master node is killed and restarted, the worker nodes would not attempt to reconnect to the Master. Therefore, when the Master node was restarted, the worker nodes needed to be restarted as well. Now, when the Master node is disconnected, the worker nodes will continuously ping the master node in attempts to reconnect to it. Once the master node restarts, it will detect one of the registration requests from its former workers. The result is that the cluster re-enters a healthy state. In addition, when the master does not receive a heartbeat from the worker, the worker was removed; however, when the worker sent a heartbeat to the master, the master used to ignore the heartbeat. Now, a master that receives a heartbeat from a worker that had been disconnected will request the worker to re-attempt the registration process, at which point the worker will send a RegisterWorker request and be re-connected accordingly. Re-connection attempts per worker are submitted every N seconds, where N is configured by the property spark.worker.reconnect.interval - this has a default of 60 seconds right now. Author: mcheah Closes #2828 from mccheah/reconnect-dead-workers and squashes the following commits: 83f8bc9 [mcheah] [SPARK-3736] More informative log message, and fixing some indentation. fe0e02f [mcheah] [SPARK-3736] Moving reconnection logic to registerWithMaster(). 94ddeca [mcheah] [SPARK-3736] Changing a log warning to a log info. a698e35 [mcheah] [SPARK-3736] Addressing PR comment to make some defs private. b9a3077 [mcheah] [SPARK-3736] Addressing PR comments related to reconnection. 2ad5ed5 [mcheah] [SPARK-3736] Cancel attempts to reconnect if the master changes. b5b34af [mcheah] [SPARK-3736] Workers reconnect when disassociated from the master. --- .../apache/spark/deploy/DeployMessage.scala | 2 + .../apache/spark/deploy/master/Master.scala | 9 ++- .../apache/spark/deploy/worker/Worker.scala | 81 ++++++++++++++----- 3 files changed, 72 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index a7368f9f3dfbe..b9dd8557ee904 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -71,6 +71,8 @@ private[deploy] object DeployMessages { case class RegisterWorkerFailed(message: String) extends DeployMessage + case class ReconnectWorker(masterUrl: String) extends DeployMessage + case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage case class LaunchExecutor( 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 f98b531316a3d..3b6bb9fe128a4 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 @@ -341,7 +341,14 @@ private[spark] class Master( case Some(workerInfo) => workerInfo.lastHeartbeat = System.currentTimeMillis() case None => - logWarning("Got heartbeat from unregistered worker " + workerId) + if (workers.map(_.id).contains(workerId)) { + logWarning(s"Got heartbeat from unregistered worker $workerId." + + " Asking it to re-register.") + sender ! ReconnectWorker(masterUrl) + } else { + logWarning(s"Got heartbeat from unregistered worker $workerId." + + " This worker was never registered, so ignoring the heartbeat.") + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 9b52cb06fb6fa..c4a8ec2e5e7b0 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -20,12 +20,14 @@ package org.apache.spark.deploy.worker import java.io.File import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date +import java.util.{UUID, Date} +import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import scala.language.postfixOps +import scala.util.Random import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} @@ -64,8 +66,22 @@ private[spark] class Worker( // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 - val REGISTRATION_TIMEOUT = 20.seconds - val REGISTRATION_RETRIES = 3 + // Model retries to connect to the master, after Hadoop's model. + // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) + // Afterwards, the next 10 attempts are between 30 and 90 seconds. + // A bit of randomness is introduced so that not all of the workers attempt to reconnect at + // the same time. + val INITIAL_REGISTRATION_RETRIES = 6 + val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 + val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 + val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { + val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits) + randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND + } + val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * + REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds + val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 + * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders @@ -103,6 +119,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + var connectionAttemptCount = 0 val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) @@ -158,7 +175,7 @@ private[spark] class Worker( connected = true } - def tryRegisterAllMasters() { + private def tryRegisterAllMasters() { for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) @@ -166,26 +183,47 @@ private[spark] class Worker( } } - def registerWithMaster() { - tryRegisterAllMasters() - var retries = 0 - registrationRetryTimer = Some { - context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - Utils.tryOrExit { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + private def retryConnectToMaster() { + Utils.tryOrExit { + connectionAttemptCount += 1 + logInfo(s"Attempting to connect to master (attempt # $connectionAttemptCount") + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = None + } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { + tryRegisterAllMasters() + if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = Some { + context.system.scheduler.schedule(PROLONGED_REGISTRATION_RETRY_INTERVAL, + PROLONGED_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) } } + } else { + logError("All masters are unresponsive! Giving up.") + System.exit(1) } } } + def registerWithMaster() { + // DisassociatedEvent may be triggered multiple times, so don't attempt registration + // if there are outstanding registration attempts scheduled. + registrationRetryTimer match { + case None => + registered = false + tryRegisterAllMasters() + connectionAttemptCount = 0 + registrationRetryTimer = Some { + context.system.scheduler.schedule(INITIAL_REGISTRATION_RETRY_INTERVAL, + INITIAL_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + } + case Some(_) => + logInfo("Not spawning another attempt to register with the master, since there is an" + + " attempt scheduled already.") + } + } + override def receiveWithLogging = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) @@ -243,6 +281,10 @@ private[spark] class Worker( System.exit(1) } + case ReconnectWorker(masterUrl) => + logInfo(s"Master with url $masterUrl requested this worker to reconnect.") + registerWithMaster() + case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_) => if (masterUrl != activeMasterUrl) { logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.") @@ -362,9 +404,10 @@ private[spark] class Worker( } } - def masterDisconnected() { + private def masterDisconnected() { logError("Connection to master failed! Waiting for master to reconnect...") connected = false + registerWithMaster() } def generateWorkerId(): String = { From eadc4c590ee43572528da55d84ed65f09153e857 Mon Sep 17 00:00:00 2001 From: Qiping Li Date: Mon, 20 Oct 2014 13:12:26 -0700 Subject: [PATCH 25/32] [SPARK-3207][MLLIB]Choose splits for continuous features in DecisionTree more adaptively DecisionTree splits on continuous features by choosing an array of values from a subsample of the data. Currently, it does not check for identical values in the subsample, so it could end up having multiple copies of the same split. In this PR, we choose splits for a continuous feature in 3 steps: 1. Sort sample values for this feature 2. Get number of occurrence of each distinct value 3. Iterate the value count array computed in step 2 to choose splits. After find splits, `numSplits` and `numBins` in metadata will be updated. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: Qiping Li Author: chouqin Author: liqi Author: qiping.lqp Closes #2780 from chouqin/dt-findsplits and squashes the following commits: 18d0301 [Qiping Li] check explicitly findsplits return distinct splits 8dc28ab [chouqin] remove blank lines ffc920f [chouqin] adjust code based on comments and add more test cases 9857039 [chouqin] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits d353596 [qiping.lqp] fix pyspark doc test 9e64699 [Qiping Li] fix random forest unit test 3c72913 [Qiping Li] fix random forest unit test 092efcb [Qiping Li] fix bug f69f47f [Qiping Li] fix bug ab303a4 [Qiping Li] fix bug af6dc97 [Qiping Li] fix bug 2a8267a [Qiping Li] fix bug c339a61 [Qiping Li] fix bug 369f812 [Qiping Li] fix style 8f46af6 [Qiping Li] add comments and unit test 9e7138e [Qiping Li] Merge branch 'dt-findsplits' of https://github.com/chouqin/spark into dt-findsplits 1b25a35 [Qiping Li] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits 0cd744a [liqi] fix bug 3652823 [Qiping Li] fix bug af7cb79 [Qiping Li] Choose splits for continuous features in DecisionTree more adaptively --- .../spark/mllib/tree/DecisionTree.scala | 104 ++++++++++++++++-- .../tree/impl/DecisionTreeMetadata.scala | 11 ++ .../spark/mllib/tree/DecisionTreeSuite.scala | 68 +++++++++++- .../spark/mllib/tree/RandomForestSuite.scala | 5 +- python/pyspark/mllib/tree.py | 4 +- 5 files changed, 176 insertions(+), 16 deletions(-) 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 03eeaa707715b..6737a2f4176c2 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 @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD @@ -909,32 +911,39 @@ object DecisionTree extends Serializable with Logging { // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - val numSplits = metadata.numSplits(featureIndex) - val numBins = metadata.numBins(featureIndex) if (metadata.isContinuous(featureIndex)) { - val numSamples = sampledInput.length + val featureSamples = sampledInput.map(lp => lp.features(featureIndex)) + val featureSplits = findSplitsForContinuousFeature(featureSamples, + metadata, featureIndex) + + val numSplits = featureSplits.length + val numBins = numSplits + 1 + logDebug(s"featureIndex = $featureIndex, numSplits = $numSplits") splits(featureIndex) = new Array[Split](numSplits) bins(featureIndex) = new Array[Bin](numBins) - val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted - val stride: Double = numSamples.toDouble / metadata.numBins(featureIndex) - logDebug("stride = " + stride) - for (splitIndex <- 0 until numSplits) { - val sampleIndex = splitIndex * stride.toInt - // Set threshold halfway in between 2 samples. - val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 + + var splitIndex = 0 + while (splitIndex < numSplits) { + val threshold = featureSplits(splitIndex) splits(featureIndex)(splitIndex) = new Split(featureIndex, threshold, Continuous, List()) + splitIndex += 1 } bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) - for (splitIndex <- 1 until numSplits) { + + splitIndex = 1 + while (splitIndex < numSplits) { bins(featureIndex)(splitIndex) = new Bin(splits(featureIndex)(splitIndex - 1), splits(featureIndex)(splitIndex), Continuous, Double.MinValue) + splitIndex += 1 } bins(featureIndex)(numSplits) = new Bin(splits(featureIndex)(numSplits - 1), new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) } else { + val numSplits = metadata.numSplits(featureIndex) + val numBins = metadata.numBins(featureIndex) // Categorical feature val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { @@ -1011,4 +1020,77 @@ object DecisionTree extends Serializable with Logging { categories } + /** + * Find splits for a continuous feature + * NOTE: Returned number of splits is set based on `featureSamples` and + * could be different from the specified `numSplits`. + * The `numSplits` attribute in the `DecisionTreeMetadata` class will be set accordingly. + * @param featureSamples feature values of each sample + * @param metadata decision tree metadata + * NOTE: `metadata.numbins` will be changed accordingly + * if there are not enough splits to be found + * @param featureIndex feature index to find splits + * @return array of splits + */ + private[tree] def findSplitsForContinuousFeature( + featureSamples: Array[Double], + metadata: DecisionTreeMetadata, + featureIndex: Int): Array[Double] = { + require(metadata.isContinuous(featureIndex), + "findSplitsForContinuousFeature can only be used to find splits for a continuous feature.") + + val splits = { + val numSplits = metadata.numSplits(featureIndex) + + // get count for each distinct value + val valueCountMap = featureSamples.foldLeft(Map.empty[Double, Int]) { (m, x) => + m + ((x, m.getOrElse(x, 0) + 1)) + } + // sort distinct values + val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray + + // if possible splits is not enough or just enough, just return all possible splits + val possibleSplits = valueCounts.length + if (possibleSplits <= numSplits) { + valueCounts.map(_._1) + } else { + // stride between splits + val stride: Double = featureSamples.length.toDouble / (numSplits + 1) + logDebug("stride = " + stride) + + // iterate `valueCount` to find splits + val splits = new ArrayBuffer[Double] + var index = 1 + // currentCount: sum of counts of values that have been visited + var currentCount = valueCounts(0)._2 + // targetCount: target value for `currentCount`. + // If `currentCount` is closest value to `targetCount`, + // then current value is a split threshold. + // After finding a split threshold, `targetCount` is added by stride. + var targetCount = stride + while (index < valueCounts.length) { + val previousCount = currentCount + currentCount += valueCounts(index)._2 + val previousGap = math.abs(previousCount - targetCount) + val currentGap = math.abs(currentCount - targetCount) + // If adding count of current value to currentCount + // makes the gap between currentCount and targetCount smaller, + // previous value is a split threshold. + if (previousGap < currentGap) { + splits.append(valueCounts(index - 1)._1) + targetCount += stride + } + index += 1 + } + + splits.toArray + } + } + + assert(splits.length > 0) + // set number of splits accordingly + metadata.setNumSplits(featureIndex, splits.length) + + splits + } } 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 index 772c02670e541..5bc0f2635c6b1 100644 --- 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 @@ -76,6 +76,17 @@ private[tree] class DecisionTreeMetadata( numBins(featureIndex) - 1 } + + /** + * Set number of splits for a continuous feature. + * For a continuous feature, number of bins is number of splits plus 1. + */ + def setNumSplits(featureIndex: Int, numSplits: Int) { + require(isContinuous(featureIndex), + s"Only number of bin for a continuous feature can be set.") + numBins(featureIndex) = numSplits + 1 + } + /** * Indicates if feature subsampling is being used. */ 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 98a72b0c4d750..8fc5e111bbc17 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 @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, Strategy} import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} @@ -102,6 +102,72 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } + test("find splits for a continuous feature") { + // find splits for normal case + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(6), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array.fill(200000)(math.random) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 5) + assert(fakeMetadata.numSplits(0) === 5) + assert(fakeMetadata.numBins(0) === 6) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // find splits should not return identical splits + // when there are not enough split candidates, reduce the number of splits in metadata + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(5), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 3) + assert(fakeMetadata.numSplits(0) === 3) + assert(fakeMetadata.numBins(0) === 4) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // find splits when most samples close to the minimum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 2) + assert(fakeMetadata.numSplits(0) === 2) + assert(fakeMetadata.numBins(0) === 3) + assert(splits(0) === 2.0) + assert(splits(1) === 3.0) + } + + // find splits when most samples close to the maximum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 1) + assert(fakeMetadata.numSplits(0) === 1) + assert(fakeMetadata.numBins(0) === 2) + assert(splits(0) === 1.0) + } + } + test("Multiclass classification with unordered categorical features:" + " split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index fb44ceb0f57ee..6b13765b98f41 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -93,8 +93,9 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 - val strategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + val strategy = new Strategy(algo = Regression, impurity = Variance, + maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + categoricalFeaturesInfo = categoricalFeaturesInfo) val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 0938eebd3a548..64ee79d83e849 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -153,9 +153,9 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, DecisionTreeModel classifier of depth 1 with 3 nodes >>> print model.toDebugString(), # it already has newline DecisionTreeModel classifier of depth 1 with 3 nodes - If (feature 0 <= 0.5) + If (feature 0 <= 0.0) Predict: 0.0 - Else (feature 0 > 0.5) + Else (feature 0 > 0.0) Predict: 1.0 >>> model.predict(array([1.0])) > 0 True From 1b3ce61ce9061719d8c28d621c3200e0bff3a4e9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 20 Oct 2014 15:29:54 -0700 Subject: [PATCH 26/32] [SPARK-3906][SQL] Adds multiple join support for SQLContext Author: Cheng Lian Closes #2767 from liancheng/multi-join and squashes the following commits: 9dc0d18 [Cheng Lian] Adds multiple join support for SQLContext --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 8 +++++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b4d606d37e732..a277684f6327c 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -181,9 +181,11 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val joinedRelation: Parser[LogicalPlan] = - relationFactor ~ joinType.? ~ (JOIN ~> relationFactor) ~ joinConditions.? ^^ { - case r1 ~ jt ~ r2 ~ cond => - Join(r1, r2, joinType = jt.getOrElse(Inner), cond) + relationFactor ~ rep1(joinType.? ~ (JOIN ~> relationFactor) ~ joinConditions.?) ^^ { + case r1 ~ joins => + joins.foldLeft(r1) { case (lhs, jt ~ rhs ~ cond) => + Join(lhs, rhs, joinType = jt.getOrElse(Inner), cond) + } } protected lazy val joinConditions: Parser[Expression] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 15f6ba4f72bbd..beb924f1715b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -720,4 +720,15 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) } + + test("Multiple join") { + checkAnswer( + sql( + """SELECT a.key, b.key, c.key + |FROM testData a + |JOIN testData b ON a.key = b.key + |JOIN testData c ON a.key = c.key + """.stripMargin), + (1 to 100).map(i => Seq(i, i, i))) + } } From e9c1afa87b8a6d9ea02aa9f7c319a36f00c0e4c3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Oct 2014 15:32:17 -0700 Subject: [PATCH 27/32] [SPARK-3800][SQL] Clean aliases from grouping expressions Author: Michael Armbrust Closes #2658 from marmbrus/nestedAggs and squashes the following commits: 862b763 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 3234521 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 8b06fdc [Michael Armbrust] possible fix for grouping on nested fields --- .../spark/sql/catalyst/analysis/Analyzer.scala | 18 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 35 insertions(+) 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 82553063145b8..a448c794213ae 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 @@ -60,6 +60,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ResolveFunctions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: + TrimAliases :: typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, @@ -89,6 +90,23 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * Removes no-op Alias expressions from the plan. + */ + object TrimAliases extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Aggregate(groups, aggs, child) => + Aggregate( + groups.map { + _ transform { + case Alias(c, _) => c + } + }, + aggs, + child) + } + } + /** * Checks for non-aggregated attributes with aggregation */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index beb924f1715b0..3959925a2e529 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -43,6 +43,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { TimeZone.setDefault(origZone) } + test("grouping on nested fields") { + jsonRDD(sparkContext.parallelize("""{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) + .registerTempTable("rows") + + checkAnswer( + sql( + """ + |select attribute, sum(cnt) + |from ( + | select nested.attribute, count(*) as cnt + | from rows + | group by nested.attribute) a + |group by attribute + """.stripMargin), + Row(1, 1) :: Nil) + } + test("SPARK-3176 Added Parser of SQL ABS()") { checkAnswer( sql("SELECT ABS(-1.3)"), From 364d52b707b5c49bc29ce60dbfe6e845a75d5d86 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 15:51:05 -0700 Subject: [PATCH 28/32] [SPARK-3966][SQL] Fix nullabilities of Cast related to DateType. Author: Takuya UESHIN Closes #2820 from ueshin/issues/SPARK-3966 and squashes the following commits: ca4a745 [Takuya UESHIN] Fix nullabilities of Cast related to DateType. --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 8e5ee12e314bf..8e5baf0eb82d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -32,6 +32,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (StringType, _: NumericType) => true case (StringType, TimestampType) => true case (StringType, DateType) => true + case (_: NumericType, DateType) => true + case (BooleanType, DateType) => true + case (DateType, _: NumericType) => true + case (DateType, BooleanType) => true case _ => child.nullable } From fce1d41611fdb27956c3394a706ed14960182a83 Mon Sep 17 00:00:00 2001 From: luogankun Date: Mon, 20 Oct 2014 16:50:51 -0700 Subject: [PATCH 29/32] [SPARK-3945]Properties of hive-site.xml is invalid in running the Thrift JDBC server Write properties of hive-site.xml to HiveContext when initilize session state in SparkSQLEnv.scala. The method of SparkSQLEnv.init() in HiveThriftServer2.scala can not write the properties of hive-site.xml to HiveContext. Such as: add configuration property spark.sql.shuffle.partititions in the hive-site.xml. Author: luogankun Closes #2800 from luogankun/SPARK-3945 and squashes the following commits: 3679efc [luogankun] [SPARK-3945]Write properties of hive-site.xml to HiveContext when initilize session state In SparkSQLEnv.scala --- .../apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 582264eb59f83..2136a2ea63543 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -39,7 +39,11 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() + @transient override lazy val sessionState = { + val state = SessionState.get() + setConf(state.getConf.getAllProperties) + state + } @transient override lazy val hiveconf = sessionState.getConf } } From 7586e2e67ad45007f78803179b04d199c174bd69 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 17:09:12 -0700 Subject: [PATCH 30/32] [SPARK-3969][SQL] Optimizer should have a super class as an interface. Some developers want to replace `Optimizer` to fit their projects but can't do so because currently `Optimizer` is an `object`. Author: Takuya UESHIN Closes #2825 from ueshin/issues/SPARK-3969 and squashes the following commits: abbc53c [Takuya UESHIN] Re-rename Optimizer object. 4d2e1bc [Takuya UESHIN] Rename Optimizer object. 9547a23 [Takuya UESHIN] Extract abstract class from Optimizer for developers to be able to replace Optimizer. --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 4 +++- .../sql/catalyst/optimizer/ExpressionOptimizationSuite.scala | 4 ++-- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) 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 3693b41404fd6..9ce7c78195830 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 @@ -28,7 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ -object Optimizer extends RuleExecutor[LogicalPlan] { +abstract class Optimizer extends RuleExecutor[LogicalPlan] + +object DefaultOptimizer extends Optimizer { val batches = Batch("Combine Limits", FixedPoint(100), CombineLimits) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index 890d6289b9dfb..ae99a3f9ba287 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -30,7 +30,7 @@ class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { expected: Any, inputRow: Row = EmptyRow): Unit = { val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) - val optimizedPlan = Optimizer(plan) + val optimizedPlan = DefaultOptimizer(plan) super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } -} \ No newline at end of file +} 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 23e7b2d270777..0e4a9ca60b00d 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types.DataType @@ -68,7 +68,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new Analyzer(catalog, functionRegistry, caseSensitive = true) @transient - protected[sql] val optimizer = Optimizer + protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer @transient protected[sql] val sqlParser = { From 0fe1c093690e5af6268182801747dfa6394dfc5b Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Mon, 20 Oct 2014 17:15:24 -0700 Subject: [PATCH 31/32] [SPARK-3940][SQL] Avoid console printing error messages three times MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If wrong sql,the console print error one times。 eg:
spark-sql> show tabless;
show tabless;
14/10/13 21:03:48 INFO ParseDriver: Parsing command: show tabless
............
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:274)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:209)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
Caused by: org.apache.hadoop.hive.ql.parse.ParseException: line 1:5 cannot recognize input near 'show' 'tabless' '' in ddl statement

	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:193)
	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:161)
	at org.apache.spark.sql.hive.HiveQl$.getAst(HiveQl.scala:218)
	at org.apache.spark.sql.hive.HiveQl$.createPlan(HiveQl.scala:226)
	... 47 more
Time taken: 4.35 seconds
14/10/13 21:03:51 INFO CliDriver: Time taken: 4.35 seconds
Author: wangxiaojing Closes #2790 from wangxiaojing/spark-3940 and squashes the following commits: e2e5c14 [wangxiaojing] sql Print the error code three times --- .../org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 7463df1f47d43..a5c457c677564 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -62,7 +62,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo } catch { case cause: Throwable => logError(s"Failed in [$command]", cause) - new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + new CommandProcessorResponse(0, ExceptionUtils.getFullStackTrace(cause), null) } } From 342b57db66e379c475daf5399baf680ff42b87c2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 20 Oct 2014 19:16:35 -0700 Subject: [PATCH 32/32] Update Building Spark link. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8dd8b70696aa2..dbf53dcd76b2d 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). +["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html). ## Interactive Scala Shell
Output OperationMeaning
print() print() Prints first ten elements of every batch of data in a DStream on the driver. - This is useful for development and debugging.
saveAsObjectFiles(prefix, [suffix])
spark.hadoop.cloneConffalseIf set to true, clones a new Hadoop Configuration object for each task. This + option should be enabled to work around Configuration thread-safety issues (see + SPARK-2546 for more details). + This is disabled by default in order to avoid unexpected performance regressions for jobs that + are not affected by these issues.
spark.executor.heartbeatInterval 10000