From 635888cbed0e3f4127252fb84db449f0cc9ed659 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 13 Jul 2014 19:27:43 -0700 Subject: [PATCH 01/34] SPARK-2363. Clean MLlib's sample data files (Just made a PR for this, mengxr was the reporter of:) MLlib has sample data under serveral folders: 1) data/mllib 2) data/ 3) mllib/data/* Per previous discussion with Matei Zaharia, we want to put them under `data/mllib` and clean outdated files. Author: Sean Owen Closes #1394 from srowen/SPARK-2363 and squashes the following commits: 54313dd [Sean Owen] Move ML example data from /mllib/data/ and /data/ into /data/mllib/ --- {mllib/data => data/mllib}/als/test.data | 0 data/{ => mllib}/kmeans_data.txt | 0 {mllib/data => data/mllib}/lr-data/random.data | 0 data/{ => mllib}/lr_data.txt | 0 data/{ => mllib}/pagerank_data.txt | 0 {mllib/data => data/mllib}/ridge-data/lpsa.data | 0 {mllib/data => data/mllib}/sample_libsvm_data.txt | 0 {mllib/data => data/mllib}/sample_naive_bayes_data.txt | 0 {mllib/data => data/mllib}/sample_svm_data.txt | 0 {mllib/data => data/mllib}/sample_tree_data.csv | 0 docs/bagel-programming-guide.md | 2 +- docs/mllib-basics.md | 6 +++--- docs/mllib-clustering.md | 4 ++-- docs/mllib-collaborative-filtering.md | 4 ++-- docs/mllib-decision-tree.md | 4 ++-- docs/mllib-linear-methods.md | 8 ++++---- docs/mllib-naive-bayes.md | 2 +- docs/mllib-optimization.md | 2 +- 18 files changed, 16 insertions(+), 16 deletions(-) rename {mllib/data => data/mllib}/als/test.data (100%) rename data/{ => mllib}/kmeans_data.txt (100%) rename {mllib/data => data/mllib}/lr-data/random.data (100%) rename data/{ => mllib}/lr_data.txt (100%) rename data/{ => mllib}/pagerank_data.txt (100%) rename {mllib/data => data/mllib}/ridge-data/lpsa.data (100%) rename {mllib/data => data/mllib}/sample_libsvm_data.txt (100%) rename {mllib/data => data/mllib}/sample_naive_bayes_data.txt (100%) rename {mllib/data => data/mllib}/sample_svm_data.txt (100%) rename {mllib/data => data/mllib}/sample_tree_data.csv (100%) diff --git a/mllib/data/als/test.data b/data/mllib/als/test.data similarity index 100% rename from mllib/data/als/test.data rename to data/mllib/als/test.data diff --git a/data/kmeans_data.txt b/data/mllib/kmeans_data.txt similarity index 100% rename from data/kmeans_data.txt rename to data/mllib/kmeans_data.txt diff --git a/mllib/data/lr-data/random.data b/data/mllib/lr-data/random.data similarity index 100% rename from mllib/data/lr-data/random.data rename to data/mllib/lr-data/random.data diff --git a/data/lr_data.txt b/data/mllib/lr_data.txt similarity index 100% rename from data/lr_data.txt rename to data/mllib/lr_data.txt diff --git a/data/pagerank_data.txt b/data/mllib/pagerank_data.txt similarity index 100% rename from data/pagerank_data.txt rename to data/mllib/pagerank_data.txt diff --git a/mllib/data/ridge-data/lpsa.data b/data/mllib/ridge-data/lpsa.data similarity index 100% rename from mllib/data/ridge-data/lpsa.data rename to data/mllib/ridge-data/lpsa.data diff --git a/mllib/data/sample_libsvm_data.txt b/data/mllib/sample_libsvm_data.txt similarity index 100% rename from mllib/data/sample_libsvm_data.txt rename to data/mllib/sample_libsvm_data.txt diff --git a/mllib/data/sample_naive_bayes_data.txt b/data/mllib/sample_naive_bayes_data.txt similarity index 100% rename from mllib/data/sample_naive_bayes_data.txt rename to data/mllib/sample_naive_bayes_data.txt diff --git a/mllib/data/sample_svm_data.txt b/data/mllib/sample_svm_data.txt similarity index 100% rename from mllib/data/sample_svm_data.txt rename to data/mllib/sample_svm_data.txt diff --git a/mllib/data/sample_tree_data.csv b/data/mllib/sample_tree_data.csv similarity index 100% rename from mllib/data/sample_tree_data.csv rename to data/mllib/sample_tree_data.csv diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index b280df0c8eeb8..7e55131754a3f 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -46,7 +46,7 @@ import org.apache.spark.bagel.Bagel._ Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it. {% highlight scala %} -val input = sc.textFile("data/pagerank_data.txt") +val input = sc.textFile("data/mllib/pagerank_data.txt") val numVerts = input.count() diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index 5796e16e8f99c..f9585251fafac 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -193,7 +193,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -val examples: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val examples: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") {% endhighlight %} @@ -207,7 +207,7 @@ import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.api.java.JavaRDD; JavaRDD examples = - MLUtils.loadLibSVMFile(jsc.sc(), "mllib/data/sample_libsvm_data.txt").toJavaRDD(); + MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt").toJavaRDD(); {% endhighlight %} @@ -218,7 +218,7 @@ examples stored in LIBSVM format. {% highlight python %} from pyspark.mllib.util import MLUtils -examples = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +examples = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") {% endhighlight %} diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 429cdf8d40cec..c76ac010d3f81 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -51,7 +51,7 @@ import org.apache.spark.mllib.clustering.KMeans import org.apache.spark.mllib.linalg.Vectors // Load and parse the data -val data = sc.textFile("data/kmeans_data.txt") +val data = sc.textFile("data/mllib/kmeans_data.txt") val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) // Cluster the data into two classes using KMeans @@ -86,7 +86,7 @@ from numpy import array from math import sqrt # Load and parse the data -data = sc.textFile("data/kmeans_data.txt") +data = sc.textFile("data/mllib/kmeans_data.txt") parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) # Build the model (cluster the data) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index d51002f015670..5cd71738722a9 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -58,7 +58,7 @@ import org.apache.spark.mllib.recommendation.ALS import org.apache.spark.mllib.recommendation.Rating // Load and parse the data -val data = sc.textFile("mllib/data/als/test.data") +val data = sc.textFile("data/mllib/als/test.data") val ratings = data.map(_.split(',') match { case Array(user, item, rate) => Rating(user.toInt, item.toInt, rate.toDouble) }) @@ -112,7 +112,7 @@ from pyspark.mllib.recommendation import ALS from numpy import array # Load and parse the data -data = sc.textFile("mllib/data/als/test.data") +data = sc.textFile("data/mllib/als/test.data") ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) # Build the recommendation model using Alternating Least Squares diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 3002a66a4fdb3..9cd768599e529 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -122,7 +122,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.impurity.Gini // Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") +val data = sc.textFile("data/mllib/sample_tree_data.csv") val parsedData = data.map { line => val parts = line.split(',').map(_.toDouble) LabeledPoint(parts(0), Vectors.dense(parts.tail)) @@ -161,7 +161,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.impurity.Variance // Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") +val data = sc.textFile("data/mllib/sample_tree_data.csv") val parsedData = data.map { line => val parts = line.split(',').map(_.toDouble) LabeledPoint(parts(0), Vectors.dense(parts.tail)) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 4dfbebbcd04b7..b4d22e0df5a85 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -187,7 +187,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils // Load training data in LIBSVM format. -val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") // Split data into training (60%) and test (40%). val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) @@ -259,7 +259,7 @@ def parsePoint(line): values = [float(x) for x in line.split(' ')] return LabeledPoint(values[0], values[1:]) -data = sc.textFile("mllib/data/sample_svm_data.txt") +data = sc.textFile("data/mllib/sample_svm_data.txt") parsedData = data.map(parsePoint) # Build the model @@ -309,7 +309,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors // Load and parse the data -val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val data = sc.textFile("data/mllib/ridge-data/lpsa.data") val parsedData = data.map { line => val parts = line.split(',') LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) @@ -356,7 +356,7 @@ def parsePoint(line): values = [float(x) for x in line.replace(',', ' ').split(' ')] return LabeledPoint(values[0], values[1:]) -data = sc.textFile("mllib/data/ridge-data/lpsa.data") +data = sc.textFile("data/mllib/ridge-data/lpsa.data") parsedData = data.map(parsePoint) # Build the model diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 1d1d7dcf6ffcb..b1650c83c98b9 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -40,7 +40,7 @@ import org.apache.spark.mllib.classification.NaiveBayes import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -val data = sc.textFile("mllib/data/sample_naive_bayes_data.txt") +val data = sc.textFile("data/mllib/sample_naive_bayes_data.txt") val parsedData = data.map { line => val parts = line.split(',') LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index ae9ede58e8e60..651958c7812f2 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -214,7 +214,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.classification.LogisticRegressionModel -val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") val numFeatures = data.take(1)(0).features.size // Split data into training (60%) and test (40%). From aab5349660109481ee944721d611771da5a93109 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 14 Jul 2014 00:42:59 -0700 Subject: [PATCH 02/34] Made rdd.py pep8 complaint by using Autopep8 and a little manual editing. Author: Prashant Sharma Closes #1354 from ScrapCodes/pep8-comp-1 and squashes the following commits: 9858ea8 [Prashant Sharma] Code Review d8851b7 [Prashant Sharma] Found # noqa works even inside comment blocks. Not sure if it works with all versions of python. 10c0cef [Prashant Sharma] Made rdd.py pep8 complaint by using Autopep8 and a little manual tweaking. --- python/pyspark/rdd.py | 150 ++++++++++++++++++++++++++---------------- 1 file changed, 92 insertions(+), 58 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f64f48e3a4c9c..0c35c666805dd 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -69,16 +69,19 @@ def _extract_concise_traceback(): file, line, fun, what = tb[0] return callsite(function=fun, file=file, linenum=line) sfile, sline, sfun, swhat = tb[first_spark_frame] - ufile, uline, ufun, uwhat = tb[first_spark_frame-1] + ufile, uline, ufun, uwhat = tb[first_spark_frame - 1] return callsite(function=sfun, file=ufile, linenum=uline) _spark_stack_depth = 0 + class _JavaStackTrace(object): + def __init__(self, sc): tb = _extract_concise_traceback() if tb is not None: - self._traceback = "%s at %s:%s" % (tb.function, tb.file, tb.linenum) + self._traceback = "%s at %s:%s" % ( + tb.function, tb.file, tb.linenum) else: self._traceback = "Error! Could not extract traceback info" self._context = sc @@ -95,7 +98,9 @@ def __exit__(self, type, value, tb): if _spark_stack_depth == 0: self._context._jsc.setCallSite(None) + class MaxHeapQ(object): + """ An implementation of MaxHeap. >>> import pyspark.rdd @@ -117,14 +122,14 @@ class MaxHeapQ(object): """ def __init__(self, maxsize): - # we start from q[1], this makes calculating children as trivial as 2 * k + # We start from q[1], so its children are always 2 * k self.q = [0] self.maxsize = maxsize def _swim(self, k): - while (k > 1) and (self.q[k/2] < self.q[k]): - self._swap(k, k/2) - k = k/2 + while (k > 1) and (self.q[k / 2] < self.q[k]): + self._swap(k, k / 2) + k = k / 2 def _swap(self, i, j): t = self.q[i] @@ -162,7 +167,9 @@ def _replaceRoot(self, value): self.q[1] = value self._sink(1) + class RDD(object): + """ A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, partitioned collection of elements that can be @@ -257,7 +264,8 @@ def map(self, f, preservesPartitioning=False): >>> sorted(rdd.map(lambda x: (x, 1)).collect()) [('a', 1), ('b', 1), ('c', 1)] """ - def func(split, iterator): return imap(f, iterator) + def func(split, iterator): + return imap(f, iterator) return PipelinedRDD(self, func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -271,7 +279,8 @@ def flatMap(self, f, preservesPartitioning=False): >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ - def func(s, iterator): return chain.from_iterable(imap(f, iterator)) + def func(s, iterator): + return chain.from_iterable(imap(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -283,7 +292,8 @@ def mapPartitions(self, f, preservesPartitioning=False): >>> rdd.mapPartitions(f).collect() [3, 7] """ - def func(s, iterator): return f(iterator) + def func(s, iterator): + return f(iterator) return self.mapPartitionsWithIndex(func) def mapPartitionsWithIndex(self, f, preservesPartitioning=False): @@ -311,17 +321,17 @@ def mapPartitionsWithSplit(self, f, preservesPartitioning=False): 6 """ warnings.warn("mapPartitionsWithSplit is deprecated; " - "use mapPartitionsWithIndex instead", DeprecationWarning, stacklevel=2) + "use mapPartitionsWithIndex instead", DeprecationWarning, stacklevel=2) return self.mapPartitionsWithIndex(f, preservesPartitioning) def getNumPartitions(self): - """ - Returns the number of partitions in RDD - >>> rdd = sc.parallelize([1, 2, 3, 4], 2) - >>> rdd.getNumPartitions() - 2 - """ - return self._jrdd.partitions().size() + """ + Returns the number of partitions in RDD + >>> rdd = sc.parallelize([1, 2, 3, 4], 2) + >>> rdd.getNumPartitions() + 2 + """ + return self._jrdd.partitions().size() def filter(self, f): """ @@ -331,7 +341,8 @@ def filter(self, f): >>> rdd.filter(lambda x: x % 2 == 0).collect() [2, 4] """ - def func(iterator): return ifilter(f, iterator) + def func(iterator): + return ifilter(f, iterator) return self.mapPartitions(func) def distinct(self): @@ -391,9 +402,11 @@ def takeSample(self, withReplacement, num, seed=None): maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) if num > maxSampleSize: - raise ValueError("Sample size cannot be greater than %d." % maxSampleSize) + raise ValueError( + "Sample size cannot be greater than %d." % maxSampleSize) - fraction = RDD._computeFractionForSampleSize(num, initialCount, withReplacement) + fraction = RDD._computeFractionForSampleSize( + num, initialCount, withReplacement) samples = self.sample(withReplacement, fraction, seed).collect() # If the first sample didn't turn out large enough, keep trying to take samples; @@ -499,17 +512,17 @@ def __add__(self, other): raise TypeError return self.union(other) - def sortByKey(self, ascending=True, numPartitions=None, keyfunc = lambda x: x): + def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. - + # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] >>> sc.parallelize(tmp).sortByKey(True, 2).collect() [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> tmp2 = [('Mary', 1), ('had', 2), ('a', 3), ('little', 4), ('lamb', 5)] >>> tmp2.extend([('whose', 6), ('fleece', 7), ('was', 8), ('white', 9)]) >>> sc.parallelize(tmp2).sortByKey(True, 3, keyfunc=lambda k: k.lower()).collect() - [('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5), ('little', 4), ('Mary', 1), ('was', 8), ('white', 9), ('whose', 6)] + [('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5),...('white', 9), ('whose', 6)] """ if numPartitions is None: numPartitions = self._defaultReducePartitions() @@ -521,10 +534,12 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc = lambda x: x): # number of (key, value) pairs falling into them if numPartitions > 1: rddSize = self.count() - maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner + # constant from Spark's RangePartitioner + maxSampleSize = numPartitions * 20.0 fraction = min(maxSampleSize / max(rddSize, 1), 1.0) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() + samples = self.sample(False, fraction, 1).map( + lambda (k, v): k).collect() samples = sorted(samples, reverse=(not ascending), key=keyfunc) # we have numPartitions many parts but one of the them has @@ -540,13 +555,13 @@ def rangePartitionFunc(k): if ascending: return p else: - return numPartitions-1-p + return numPartitions - 1 - p def mapFunc(iterator): yield sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) return (self.partitionBy(numPartitions, partitionFunc=rangePartitionFunc) - .mapPartitions(mapFunc,preservesPartitioning=True) + .mapPartitions(mapFunc, preservesPartitioning=True) .flatMap(lambda x: x, preservesPartitioning=True)) def sortBy(self, keyfunc, ascending=True, numPartitions=None): @@ -570,7 +585,8 @@ def glom(self): >>> sorted(rdd.glom().collect()) [[1, 2], [3, 4]] """ - def func(iterator): yield list(iterator) + def func(iterator): + yield list(iterator) return self.mapPartitions(func) def cartesian(self, other): @@ -607,7 +623,9 @@ def pipe(self, command, env={}): ['1', '2', '', '3'] """ def func(iterator): - pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) + pipe = Popen( + shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) + def pipe_objs(out): for obj in iterator: out.write(str(obj).rstrip('\n') + '\n') @@ -646,7 +664,7 @@ def collect(self): Return a list that contains all of the elements in this RDD. """ with _JavaStackTrace(self.context) as st: - bytesInJava = self._jrdd.collect().iterator() + bytesInJava = self._jrdd.collect().iterator() return list(self._collect_iterator_through_file(bytesInJava)) def _collect_iterator_through_file(self, iterator): @@ -736,7 +754,6 @@ def func(iterator): return self.mapPartitions(func).fold(zeroValue, combOp) - def max(self): """ Find the maximum item in this RDD. @@ -844,6 +861,7 @@ def countPartition(iterator): for obj in iterator: counts[obj] += 1 yield counts + def mergeMaps(m1, m2): for (k, v) in m2.iteritems(): m1[k] += v @@ -888,22 +906,22 @@ def takeOrdered(self, num, key=None): def topNKeyedElems(iterator, key_=None): q = MaxHeapQ(num) for k in iterator: - if key_ != None: + if key_ is not None: k = (key_(k), k) q.insert(k) yield q.getElements() def unKey(x, key_=None): - if key_ != None: + if key_ is not None: x = [i[1] for i in x] return x def merge(a, b): return next(topNKeyedElems(a + b)) - result = self.mapPartitions(lambda i: topNKeyedElems(i, key)).reduce(merge) + result = self.mapPartitions( + lambda i: topNKeyedElems(i, key)).reduce(merge) return sorted(unKey(result, key), key=key) - def take(self, num): """ Take the first num elements of the RDD. @@ -947,7 +965,8 @@ def takeUpToNumLeft(iterator): yield next(iterator) taken += 1 - p = range(partsScanned, min(partsScanned + numPartsToTry, totalParts)) + p = range( + partsScanned, min(partsScanned + numPartsToTry, totalParts)) res = self.context.runJob(self, takeUpToNumLeft, p, True) items += res @@ -977,7 +996,7 @@ def saveAsPickleFile(self, path, batchSize=10): [1, 2, 'rdd', 'spark'] """ self._reserialize(BatchedSerializer(PickleSerializer(), - batchSize))._jrdd.saveAsObjectFile(path) + batchSize))._jrdd.saveAsObjectFile(path) def saveAsTextFile(self, path): """ @@ -1075,6 +1094,7 @@ def reducePartition(iterator): for (k, v) in iterator: m[k] = v if k not in m else func(m[k], v) yield m + def mergeMaps(m1, m2): for (k, v) in m2.iteritems(): m1[k] = v if k not in m1 else func(m1[k], v) @@ -1162,6 +1182,7 @@ def partitionBy(self, numPartitions, partitionFunc=None): # form the hash buckets in Python, transferring O(numPartitions) objects # to Java. Each object is a (splitNumber, [objects]) pair. outputSerializer = self.ctx._unbatched_serializer + def add_shuffle_key(split, iterator): buckets = defaultdict(list) @@ -1174,7 +1195,8 @@ def add_shuffle_key(split, iterator): keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True with _JavaStackTrace(self.context) as st: - pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() + pairRDD = self.ctx._jvm.PairwiseRDD( + keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() @@ -1213,6 +1235,7 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, """ if numPartitions is None: numPartitions = self._defaultReducePartitions() + def combineLocally(iterator): combiners = {} for x in iterator: @@ -1224,10 +1247,11 @@ def combineLocally(iterator): return combiners.iteritems() locally_combined = self.mapPartitions(combineLocally) shuffled = locally_combined.partitionBy(numPartitions) + def _mergeCombiners(iterator): combiners = {} for (k, v) in iterator: - if not k in combiners: + if k not in combiners: combiners[k] = v else: combiners[k] = mergeCombiners(combiners[k], v) @@ -1236,17 +1260,19 @@ def _mergeCombiners(iterator): def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None): """ - Aggregate the values of each key, using given combine functions and a neutral "zero value". - This function can return a different result type, U, than the type of the values in this RDD, - V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, - The former operation is used for merging values within a partition, and the latter is used - for merging values between partitions. To avoid memory allocation, both of these functions are + Aggregate the values of each key, using given combine functions and a neutral + "zero value". This function can return a different result type, U, than the type + of the values in this RDD, V. Thus, we need one operation for merging a V into + a U and one operation for merging two U's, The former operation is used for merging + values within a partition, and the latter is used for merging values between + partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U. """ def createZero(): - return copy.deepcopy(zeroValue) + return copy.deepcopy(zeroValue) - return self.combineByKey(lambda v: seqFunc(createZero(), v), seqFunc, combFunc, numPartitions) + return self.combineByKey( + lambda v: seqFunc(createZero(), v), seqFunc, combFunc, numPartitions) def foldByKey(self, zeroValue, func, numPartitions=None): """ @@ -1261,11 +1287,10 @@ def foldByKey(self, zeroValue, func, numPartitions=None): [('a', 2), ('b', 1)] """ def createZero(): - return copy.deepcopy(zeroValue) + return copy.deepcopy(zeroValue) return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions) - # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ @@ -1292,7 +1317,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultIterable(x)) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): @@ -1362,7 +1387,8 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - filter_func = lambda (key, vals): len(vals[0]) > 0 and len(vals[1]) == 0 + def filter_func((key, vals)): + return len(vals[0]) > 0 and len(vals[1]) == 0 map_func = lambda (key, vals): [(key, val) for val in vals[0]] return self.cogroup(other, numPartitions).filter(filter_func).flatMap(map_func) @@ -1375,8 +1401,9 @@ def subtract(self, other, numPartitions=None): >>> sorted(x.subtract(y).collect()) [('a', 1), ('b', 4), ('b', 5)] """ - rdd = other.map(lambda x: (x, True)) # note: here 'True' is just a placeholder - return self.map(lambda x: (x, True)).subtractByKey(rdd).map(lambda tpl: tpl[0]) # note: here 'True' is just a placeholder + # note: here 'True' is just a placeholder + rdd = other.map(lambda x: (x, True)) + return self.map(lambda x: (x, True)).subtractByKey(rdd).map(lambda tpl: tpl[0]) def keyBy(self, f): """ @@ -1434,7 +1461,7 @@ def zip(self, other): """ pairRDD = self._jrdd.zip(other._jrdd) deserializer = PairDeserializer(self._jrdd_deserializer, - other._jrdd_deserializer) + other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) def name(self): @@ -1503,7 +1530,9 @@ def _defaultReducePartitions(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. + class PipelinedRDD(RDD): + """ Pipelined maps: >>> rdd = sc.parallelize([1, 2, 3, 4]) @@ -1519,6 +1548,7 @@ class PipelinedRDD(RDD): >>> rdd.flatMap(lambda x: [x, x]).reduce(add) 20 """ + def __init__(self, prev, func, preservesPartitioning=False): if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable(): # This transformation is the first in its stage: @@ -1528,6 +1558,7 @@ def __init__(self, prev, func, preservesPartitioning=False): self._prev_jrdd_deserializer = prev._jrdd_deserializer else: prev_func = prev.func + def pipeline_func(split, iterator): return func(split, prev_func(split, iterator)) self.func = pipeline_func @@ -1560,11 +1591,13 @@ def _jrdd(self): env = MapConverter().convert(self.ctx.environment, self.ctx._gateway._gateway_client) includes = ListConverter().convert(self.ctx._python_includes, - self.ctx._gateway._gateway_client) + self.ctx._gateway._gateway_client) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), - bytearray(pickled_command), env, includes, self.preservesPartitioning, - self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, - class_tag) + bytearray(pickled_command), + env, includes, self.preservesPartitioning, + self.ctx.pythonExec, + broadcast_vars, self.ctx._javaAccumulator, + class_tag) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val @@ -1579,7 +1612,8 @@ def _test(): # The small batch size here ensures that we see multiple batches, # even in these small test examples: globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) From 38ccd6ebd412cfbf82ae9d8a0998ff697db11455 Mon Sep 17 00:00:00 2001 From: Daoyuan Date: Mon, 14 Jul 2014 10:40:44 -0700 Subject: [PATCH 03/34] move some test file to match src code Just move some test suite to corresponding package Author: Daoyuan Closes #1401 from adrian-wang/movetestfiles and squashes the following commits: d1a6803 [Daoyuan] move some test file to match src code --- .../spark/{ => broadcast}/BroadcastSuite.scala | 8 +++----- .../{ => network}/ConnectionManagerSuite.scala | 11 +++++------ .../org/apache/spark/{ => rdd}/PipedRDDSuite.scala | 14 +++++--------- .../spark/{ => rdd}/ZippedPartitionsSuite.scala | 3 ++- .../apache/spark/{ => util}/AkkaUtilsSuite.scala | 8 ++++---- 5 files changed, 19 insertions(+), 25 deletions(-) rename core/src/test/scala/org/apache/spark/{ => broadcast}/BroadcastSuite.scala (98%) rename core/src/test/scala/org/apache/spark/{ => network}/ConnectionManagerSuite.scala (97%) rename core/src/test/scala/org/apache/spark/{ => rdd}/PipedRDDSuite.scala (95%) rename core/src/test/scala/org/apache/spark/{ => rdd}/ZippedPartitionsSuite.scala (95%) rename core/src/test/scala/org/apache/spark/{ => util}/AkkaUtilsSuite.scala (99%) diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/BroadcastSuite.scala rename to core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index c9936256a5b95..7c3d0208b195a 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -15,14 +15,12 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.broadcast +import org.apache.spark.storage.{BroadcastBlockId, _} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.scalatest.FunSuite -import org.apache.spark.storage._ -import org.apache.spark.broadcast.{Broadcast, HttpBroadcast} -import org.apache.spark.storage.BroadcastBlockId - class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala index df6b2604c8d8a..415ad8c432c12 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala @@ -15,15 +15,14 @@ * limitations under the License. */ -package org.apache.spark - -import org.scalatest.FunSuite +package org.apache.spark.network import java.nio._ -import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId} -import scala.concurrent.Await -import scala.concurrent.TimeoutException +import org.apache.spark.{SecurityManager, SparkConf} +import org.scalatest.FunSuite + +import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ import scala.language.postfixOps diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/PipedRDDSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index db56a4acdd6f5..be972c5e97a7e 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -15,25 +15,21 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.io.File -import org.scalatest.FunSuite - -import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} -import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} +import org.apache.spark._ +import org.scalatest.FunSuite import scala.collection.Map import scala.language.postfixOps import scala.sys.process._ import scala.util.Try -import org.apache.hadoop.io.{Text, LongWritable} - -import org.apache.spark.executor.TaskMetrics - class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { diff --git a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala index 4f87fd8654c4a..72596e86865b2 100644 --- a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd +import org.apache.spark.SharedSparkContext import org.scalatest.FunSuite object ZippedPartitionsSuite { diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala rename to core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 4ab870e751778..c4765e53de17b 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark - -import org.scalatest.FunSuite +package org.apache.spark.util import akka.actor._ +import org.apache.spark._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.AkkaUtils +import org.scalatest.FunSuite + import scala.concurrent.Await /** From d60b09bb60cff106fa0acddebf35714503b20f03 Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Mon, 14 Jul 2014 13:22:24 -0700 Subject: [PATCH 04/34] [SPARK-2443][SQL] Fix slow read from partitioned tables This fix obtains a comparable performance boost as [PR #1390](https://github.com/apache/spark/pull/1390) by moving an array update and deserializer initialization out of a potentially very long loop. Suggested by yhuai. The below results are updated for this fix. ## Benchmarks Generated a local text file with 10M rows of simple key-value pairs. The data is loaded as a table through Hive. Results are obtained on my local machine using hive/console. Without the fix: Type | Non-partitioned | Partitioned (1 part) ------------ | ------------ | ------------- First run | 9.52s end-to-end (1.64s Spark job) | 36.6s (28.3s) Stablized runs | 1.21s (1.18s) | 27.6s (27.5s) With this fix: Type | Non-partitioned | Partitioned (1 part) ------------ | ------------ | ------------- First run | 9.57s (1.46s) | 11.0s (1.69s) Stablized runs | 1.13s (1.10s) | 1.23s (1.19s) Author: Zongheng Yang Closes #1408 from concretevitamin/slow-read-2 and squashes the following commits: d86e437 [Zongheng Yang] Move update & initialization out of potentially long loop. --- .../scala/org/apache/spark/sql/hive/TableReader.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 8cfde46186ca4..c3942578d6b5a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -164,13 +164,17 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon hivePartitionRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val rowWithPartArr = new Array[Object](2) + + // The update and deserializer initialization are intentionally + // kept out of the below iter.map loop to save performance. + rowWithPartArr.update(1, partValues) + val deserializer = localDeserializer.newInstance() + deserializer.initialize(hconf, partProps) + // Map each tuple to a row object iter.map { value => - val deserializer = localDeserializer.newInstance() - deserializer.initialize(hconf, partProps) val deserializedRow = deserializer.deserialize(value) rowWithPartArr.update(0, deserializedRow) - rowWithPartArr.update(1, partValues) rowWithPartArr.asInstanceOf[Object] } } From 3dd8af7a6623201c28231f4b71f59ea4e9ae29bf Mon Sep 17 00:00:00 2001 From: li-zhihui Date: Mon, 14 Jul 2014 15:32:49 -0500 Subject: [PATCH 05/34] [SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered --- .../scala/org/apache/spark/SparkContext.scala | 11 ++++- .../spark/scheduler/SchedulerBackend.scala | 1 + .../spark/scheduler/TaskSchedulerImpl.scala | 15 +++++++ .../CoarseGrainedSchedulerBackend.scala | 29 ++++++++++++++ .../cluster/SparkDeploySchedulerBackend.scala | 1 + docs/configuration.md | 19 +++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 1 + .../yarn/ApplicationMasterArguments.scala | 6 ++- .../cluster/YarnClientClusterScheduler.scala | 2 + .../cluster/YarnClientSchedulerBackend.scala | 1 + .../cluster/YarnClusterScheduler.scala | 2 + .../cluster/YarnClusterSchedulerBackend.scala | 40 +++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 1 + 13 files changed, 127 insertions(+), 2 deletions(-) create mode 100644 yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8819e73d17fb2..8052499ab7526 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1531,7 +1531,16 @@ object SparkContext extends Logging { throw new SparkException("YARN mode not available ?", e) } } - val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + val backend = try { + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend") + val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) + cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) + } + } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 6a6d8e609bc39..e41e0a9841691 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -30,4 +30,5 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException + def isReady(): Boolean = true } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5ed2803d76afc..4b6d6da5a6e61 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -145,6 +145,10 @@ private[spark] class TaskSchedulerImpl( } } + override def postStartHook() { + waitBackendReady() + } + override def submitTasks(taskSet: TaskSet) { val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") @@ -437,6 +441,17 @@ private[spark] class TaskSchedulerImpl( // By default, rack is unknown def getRackForHost(value: String): Option[String] = None + + private def waitBackendReady(): Unit = { + if (backend.isReady) { + return + } + while (!backend.isReady) { + synchronized { + this.wait(100) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 05d01b0c821f9..0f5545e2ed65f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -46,9 +46,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + var totalExpectedExecutors = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + // Submit tasks only after (registered executors / total expected executors) + // is equal to at least this value, that is double between 0 and 1. + var minRegisteredRatio = conf.getDouble("spark.scheduler.minRegisteredExecutorsRatio", 0) + if (minRegisteredRatio > 1) minRegisteredRatio = 1 + // Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the time(milliseconds). + val maxRegisteredWaitingTime = + conf.getInt("spark.scheduler.maxRegisteredExecutorsWaitingTime", 30000) + val createTime = System.currentTimeMillis() + var ready = if (minRegisteredRatio <= 0) true else false class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -83,6 +93,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) + if (executorActor.size >= totalExpectedExecutors.get() * minRegisteredRatio && !ready) { + ready = true + logInfo("SchedulerBackend is ready for scheduling beginning, registered executors: " + + executorActor.size + ", total expected executors: " + totalExpectedExecutors.get() + + ", minRegisteredExecutorsRatio: " + minRegisteredRatio) + } makeOffers() } @@ -247,6 +263,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A throw new SparkException("Error notifying standalone scheduler's driver actor", e) } } + + override def isReady(): Boolean = { + if (ready) { + return true + } + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + ready = true + logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + + "maxRegisteredExecutorsWaitingTime: " + maxRegisteredWaitingTime) + return true + } + false + } } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9c07b3f7b695a..bf2dc88e29048 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -95,6 +95,7 @@ private[spark] class SparkDeploySchedulerBackend( override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { + totalExpectedExecutors.addAndGet(1) logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( fullId, hostPort, cores, Utils.megabytesToString(memory))) } diff --git a/docs/configuration.md b/docs/configuration.md index 0aea23ab59502..07aa4c035446b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -699,6 +699,25 @@ Apart from these, the following properties are also available, and may be useful (in milliseconds) + + spark.scheduler.minRegisteredExecutorsRatio + 0 + + The minimum ratio of registered executors (registered executors / total expected executors) + to wait for before scheduling begins. Specified as a double between 0 and 1. + Regardless of whether the minimum ratio of executors has been reached, + the maximum amount of time it will wait before scheduling begins is controlled by config + spark.scheduler.maxRegisteredExecutorsWaitingTime + + + + spark.scheduler.maxRegisteredExecutorsWaitingTime + 30000 + + Maximum amount of time to wait for executors to register before scheduling begins + (in milliseconds). + + #### Security diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 438737f7a6b60..062f946a9fe93 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -184,6 +184,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") + System.setProperty("spark.executor.instances", args.numExecutors.toString) val mainMethod = Class.forName( args.userClass, false /* initialize */ , diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 25cc9016b10a6..4c383ab574abe 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -26,7 +26,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 - var numExecutors = 2 + var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) @@ -93,3 +93,7 @@ class ApplicationMasterArguments(val args: Array[String]) { System.exit(exitCode) } } + +object ApplicationMasterArguments { + val DEFAULT_NUMBER_EXECUTORS = 2 +} diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 6b91e6b9eb899..15e8c21aa5906 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -40,8 +40,10 @@ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configur override def postStartHook() { + super.postStartHook() // The yarn application is running, but the executor might not yet ready // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + // TODO It needn't after waitBackendReady Thread.sleep(2000L) logInfo("YarnClientClusterScheduler.postStartHook done") } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index fd2694fe7278d..0f9fdcfcb6510 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -75,6 +75,7 @@ private[spark] class YarnClientSchedulerBackend( logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) + totalExpectedExecutors.set(args.numExecutors) client = new Client(args, conf) appId = client.runApp() waitForApp() diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 39cdd2e8a522b..9ee53d797c8ea 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -48,9 +48,11 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) override def postStartHook() { val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) + super.postStartHook() if (sparkContextInitialized){ ApplicationMaster.waitForInitialAllocations() // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + // TODO It needn't after waitBackendReady Thread.sleep(3000L) } logInfo("YarnClusterScheduler.postStartHook done") diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..a04b08f43cc5a --- /dev/null +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -0,0 +1,40 @@ +/* + * 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.scheduler.cluster + +import org.apache.spark.SparkContext +import org.apache.spark.deploy.yarn.ApplicationMasterArguments +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.util.IntParam + +private[spark] class YarnClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + + override def start() { + super.start() + var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) { + numExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")).getOrElse(numExecutors) + } + // System property can override environment variable. + numExecutors = sc.getConf.getInt("spark.executor.instances", numExecutors) + totalExpectedExecutors.set(numExecutors) + } +} diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index ee1e9c9c23d22..1a24ec759b546 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -164,6 +164,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") + System.setProperty("spark.executor.instances", args.numExecutors.toString) val mainMethod = Class.forName( args.userClass, false, From 9fe693b5b6ed6af34ee1e800ab89c8a11991ea38 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 14 Jul 2014 15:42:28 -0700 Subject: [PATCH 06/34] [SPARK-2446][SQL] Add BinaryType support to Parquet I/O. Note that this commit changes the semantics when loading in data that was created with prior versions of Spark SQL. Before, we were writing out strings as Binary data without adding any other annotations. Thus, when data is read in from prior versions, data that was StringType will now become BinaryType. Users that need strings can CAST that column to a String. It was decided that while this breaks compatibility, it does make us compatible with other systems (Hive, Thrift, etc) and adds support for Binary data, so this is the right decision long term. To support `BinaryType`, the following changes are needed: - Make `StringType` use `OriginalType.UTF8` - Add `BinaryType` using `PrimitiveTypeName.BINARY` without `OriginalType` Author: Takuya UESHIN Closes #1373 from ueshin/issues/SPARK-2446 and squashes the following commits: ecacb92 [Takuya UESHIN] Add BinaryType support to Parquet I/O. 616e04a [Takuya UESHIN] Make StringType use OriginalType.UTF8. --- .../spark/sql/parquet/ParquetConverter.scala | 2 +- .../sql/parquet/ParquetTableSupport.scala | 4 ++ .../spark/sql/parquet/ParquetTestData.scala | 18 +++--- .../spark/sql/parquet/ParquetTypes.scala | 62 ++++++++++--------- .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +++-- 5 files changed, 57 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 75748b2b54400..de8fe2dae38f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -114,7 +114,7 @@ private[sql] object CatalystConverter { } } // All other primitive types use the default converter - case ctype: NativeType => { // note: need the type tag here! + case ctype: PrimitiveType => { // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) } case _ => throw new RuntimeException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 108f8b6815423..f1953a008a49b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -191,6 +191,8 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { value.asInstanceOf[String].getBytes("utf-8") ) ) + case BinaryType => writer.addBinary( + Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) case ShortType => writer.addInteger(value.asInstanceOf[Short]) case LongType => writer.addLong(value.asInstanceOf[Long]) @@ -299,6 +301,8 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { record(index).asInstanceOf[String].getBytes("utf-8") ) ) + case BinaryType => writer.addBinary( + Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) case ShortType => writer.addInteger(record.getShort(index)) case LongType => writer.addLong(record.getLong(index)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 1dc58633a2a68..d4599da711254 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -58,7 +58,7 @@ private[sql] object ParquetTestData { """message myrecord { optional boolean myboolean; optional int32 myint; - optional binary mystring; + optional binary mystring (UTF8); optional int64 mylong; optional float myfloat; optional double mydouble; @@ -87,7 +87,7 @@ private[sql] object ParquetTestData { message myrecord { required boolean myboolean; required int32 myint; - required binary mystring; + required binary mystring (UTF8); required int64 mylong; required float myfloat; required double mydouble; @@ -119,14 +119,14 @@ private[sql] object ParquetTestData { // so that array types can be translated correctly. """ message AddressBook { - required binary owner; + required binary owner (UTF8); optional group ownerPhoneNumbers { - repeated binary array; + repeated binary array (UTF8); } optional group contacts { repeated group array { - required binary name; - optional binary phoneNumber; + required binary name (UTF8); + optional binary phoneNumber (UTF8); } } } @@ -181,16 +181,16 @@ private[sql] object ParquetTestData { required int32 x; optional group data1 { repeated group map { - required binary key; + required binary key (UTF8); required int32 value; } } required group data2 { repeated group map { - required binary key; + required binary key (UTF8); required group value { required int64 payload1; - optional binary payload2; + optional binary payload2 (UTF8); } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index f9046368e7ced..7f6ad908f78ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -42,20 +42,22 @@ private[parquet] object ParquetTypesConverter extends Logging { def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass - def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { - case ParquetPrimitiveTypeName.BINARY => StringType - case ParquetPrimitiveTypeName.BOOLEAN => BooleanType - case ParquetPrimitiveTypeName.DOUBLE => DoubleType - case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) - case ParquetPrimitiveTypeName.FLOAT => FloatType - case ParquetPrimitiveTypeName.INT32 => IntegerType - case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => - // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? - sys.error("Potential loss of precision: cannot convert INT96") - case _ => sys.error( - s"Unsupported parquet datatype $parquetType") - } + def toPrimitiveDataType(parquetType: ParquetPrimitiveType): DataType = + parquetType.getPrimitiveTypeName match { + case ParquetPrimitiveTypeName.BINARY + if parquetType.getOriginalType == ParquetOriginalType.UTF8 => StringType + case ParquetPrimitiveTypeName.BINARY => BinaryType + case ParquetPrimitiveTypeName.BOOLEAN => BooleanType + case ParquetPrimitiveTypeName.DOUBLE => DoubleType + case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 => IntegerType + case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 => + // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? + sys.error("Potential loss of precision: cannot convert INT96") + case _ => sys.error( + s"Unsupported parquet datatype $parquetType") + } /** * Converts a given Parquet `Type` into the corresponding @@ -104,7 +106,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } if (parquetType.isPrimitive) { - toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + toPrimitiveDataType(parquetType.asPrimitiveType) } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { @@ -164,18 +166,17 @@ private[parquet] object ParquetTypesConverter extends Logging { * @return The name of the corresponding Parquet primitive type */ def fromPrimitiveDataType(ctype: DataType): - Option[ParquetPrimitiveTypeName] = ctype match { - case StringType => Some(ParquetPrimitiveTypeName.BINARY) - case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) - case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) - case ArrayType(ByteType) => - Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) - case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) - case IntegerType => Some(ParquetPrimitiveTypeName.INT32) + Option[(ParquetPrimitiveTypeName, Option[ParquetOriginalType])] = ctype match { + case StringType => Some(ParquetPrimitiveTypeName.BINARY, Some(ParquetOriginalType.UTF8)) + case BinaryType => Some(ParquetPrimitiveTypeName.BINARY, None) + case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN, None) + case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE, None) + case FloatType => Some(ParquetPrimitiveTypeName.FLOAT, None) + case IntegerType => Some(ParquetPrimitiveTypeName.INT32, None) // There is no type for Byte or Short so we promote them to INT32. - case ShortType => Some(ParquetPrimitiveTypeName.INT32) - case ByteType => Some(ParquetPrimitiveTypeName.INT32) - case LongType => Some(ParquetPrimitiveTypeName.INT64) + case ShortType => Some(ParquetPrimitiveTypeName.INT32, None) + case ByteType => Some(ParquetPrimitiveTypeName.INT32, None) + case LongType => Some(ParquetPrimitiveTypeName.INT64, None) case _ => None } @@ -227,9 +228,10 @@ private[parquet] object ParquetTypesConverter extends Logging { if (nullable) Repetition.OPTIONAL else Repetition.REQUIRED } val primitiveType = fromPrimitiveDataType(ctype) - if (primitiveType.isDefined) { - new ParquetPrimitiveType(repetition, primitiveType.get, name) - } else { + primitiveType.map { + case (primitiveType, originalType) => + new ParquetPrimitiveType(repetition, primitiveType, name, originalType.orNull) + }.getOrElse { ctype match { case ArrayType(elementType) => { val parquetElementType = fromDataType( @@ -237,7 +239,7 @@ private[parquet] object ParquetTypesConverter extends Logging { CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, nullable = false, inArray = true) - ConversionPatterns.listType(repetition, name, parquetElementType) + ConversionPatterns.listType(repetition, name, parquetElementType) } case StructType(structFields) => { val fields = structFields.map { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 8fa143e2deca6..3c911e9a4e7b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -65,7 +65,8 @@ case class AllDataTypes( doubleField: Double, shortField: Short, byteField: Byte, - booleanField: Boolean) + booleanField: Boolean, + binaryField: Array[Byte]) case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -76,6 +77,7 @@ case class AllDataTypesWithNonPrimitiveType( shortField: Short, byteField: Byte, booleanField: Boolean, + binaryField: Array[Byte], array: Seq[Int], map: Map[Int, String], data: Data) @@ -116,7 +118,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) TestSQLContext.sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + (0 to x).map(_.toByte).toArray)) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -129,6 +132,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getShort(5) === i.toShort) assert(result(i).getByte(6) === i.toByte) assert(result(i).getBoolean(7) === (i % 2 == 0)) + assert(result(i)(8) === (0 to i).map(_.toByte).toArray) } } @@ -138,6 +142,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.sparkContext.parallelize(range) .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + (0 to x).map(_.toByte).toArray, (0 until x), (0 until x).map(i => i -> s"$i").toMap, Data((0 until x), Nested(x, s"$x")))) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() @@ -151,9 +156,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getShort(5) === i.toShort) assert(result(i).getByte(6) === i.toByte) assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === (0 until i)) - assert(result(i)(9) === (0 until i).map(i => i -> s"$i").toMap) - assert(result(i)(10) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) + assert(result(i)(8) === (0 to i).map(_.toByte).toArray) + assert(result(i)(9) === (0 until i)) + assert(result(i)(10) === (0 until i).map(i => i -> s"$i").toMap) + assert(result(i)(11) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) } } From e2255e4b2c404f31ac9f7af9ed445141af980973 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 14 Jul 2014 23:06:35 -0700 Subject: [PATCH 07/34] [SPARK-2467] Revert SparkBuild to publish-local to both .m2 and .ivy2. Author: Takuya UESHIN Closes #1398 from ueshin/issues/SPARK-2467 and squashes the following commits: 7f01d58 [Takuya UESHIN] Revert SparkBuild to publish-local to both .m2 and .ivy2. --- project/SparkBuild.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 44abbc152f99f..754d54e89361f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -19,6 +19,7 @@ import scala.util.Properties import scala.collection.JavaConversions._ import sbt._ +import sbt.Classpaths.publishTask import sbt.Keys._ import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} @@ -103,12 +104,23 @@ object SparkBuild extends PomBuild { override val userPropertiesMap = System.getProperties.toMap + lazy val MavenCompile = config("m2r") extend(Compile) + lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") + lazy val sharedSettings = graphSettings ++ ScalaStyleSettings ++ Seq ( javaHome := Properties.envOrNone("JAVA_HOME").map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", - publishMavenStyle := true + publishMavenStyle := true, + + otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))), + publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map { + (arts, _, level) => new PublishConfiguration(None, "dotM2", arts, Seq(), level) + }, + publishMavenStyle in MavenCompile := true, + publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), + publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) /** Following project only exists to pull previous artifacts of Spark for generating From 1f99fea53b5ff994dd4a12b44625d35186e269ff Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 14 Jul 2014 23:09:13 -0700 Subject: [PATCH 08/34] SPARK-2486: Utils.getCallSite is now resilient to bogus frames When running Spark under certain instrumenting profilers, Utils.getCallSite could crash with an NPE. This commit makes it more resilient to failures occurring while inspecting stack frames. Author: William Benton Closes #1413 from willb/spark-2486 and squashes the following commits: b7c0274 [William Benton] Use explicit null checks instead of Try() 0f0c1ae [William Benton] Utils.getCallSite is now resilient to bogus frames --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 a2454e120a8ab..d72c97bbe816a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -809,7 +809,11 @@ private[spark] object Utils extends Logging { */ def getCallSite: CallSite = { val trace = Thread.currentThread.getStackTrace() - .filterNot(_.getMethodName.contains("getStackTrace")) + .filterNot((ste:StackTraceElement) => + // When running under some profilers, the current stack trace might contain some bogus + // frames. This is intended to ensure that we don't crash in these situations by + // ignoring any frames that we can't examine. + (ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace"))) // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD From a2aa7bebae31e1e7ec23d31aaa436283743b283b Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 14 Jul 2014 23:38:12 -0700 Subject: [PATCH 09/34] Add/increase severity of warning in documentation of groupBy() groupBy()/groupByKey() is notorious for being a very convenient API that can lead to poor performance when used incorrectly. This PR just makes it clear that users should be cautious not to rely on this API when they really want a different (more performant) one, such as reduceByKey(). (Note that one source of confusion is the name; this groupBy() is not the same as a SQL GROUP-BY, which is used for aggregation and is more similar in nature to Spark's reduceByKey().) Author: Aaron Davidson Closes #1380 from aarondav/warning and squashes the following commits: f60da39 [Aaron Davidson] Give better advice d0afb68 [Aaron Davidson] Add/increase severity of warning in documentation of groupBy() --- .../apache/spark/rdd/PairRDDFunctions.scala | 18 +++++++++--------- .../main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++++++++ 2 files changed, 21 insertions(+), 9 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 fc9beb166befe..9d62d53fcb23f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -353,9 +353,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance. + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not @@ -373,9 +373,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance. + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) @@ -462,9 +462,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance, + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 4e841bc992bff..a25f263bea5c1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -509,6 +509,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) @@ -516,6 +520,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) @@ -523,6 +531,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) : RDD[(K, Iterable[T])] = { From c6d75745de58ff1445912bf72a58b6ad2b3f863c Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 14 Jul 2014 23:55:39 -0700 Subject: [PATCH 10/34] [SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS When running jobs with YARN Cluster mode and using HistoryServer, the files in the Staging Directory (~/.sparkStaging on HDFS) cannot be deleted. HistoryServer uses directory where event log is written, and the directory is represented as a instance of o.a.h.f.FileSystem created by using FileSystem.get. On the other hand, ApplicationMaster has a instance named fs, which also created by using FileSystem.get. FileSystem.get returns cached same instance when URI passed to the method represents same file system and the method is called by same user. Because of the behavior, when the directory for event log is on HDFS, fs of ApplicationMaster and fileSystem of FileLogger is same instance. When shutting down ApplicationMaster, fileSystem.close is called in FileLogger#stop, which is invoked by SparkContext#stop indirectly. And ApplicationMaster#cleanupStagingDir also called by JVM shutdown hook. In this method, fs.delete(stagingDirPath) is invoked. Because fs.delete in ApplicationMaster is called after fileSystem.close in FileLogger, fs.delete fails and results not deleting files in the staging directory. I think, calling fileSystem.delete is not needed. Author: Kousuke Saruta Closes #1326 from sarutak/SPARK-2390 and squashes the following commits: 10e1a88 [Kousuke Saruta] Removed fileSystem.close from FileLogger.scala not to prevent any other FileSystem operation --- core/src/main/scala/org/apache/spark/util/FileLogger.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 6a95dc06e155d..9dcdafdd6350e 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -196,6 +196,5 @@ private[spark] class FileLogger( def stop() { hadoopDataStream.foreach(_.close()) writer.foreach(_.close()) - fileSystem.close() } } From c7c7ac83392b10abb011e6aead1bf92e7c73695e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 15 Jul 2014 00:13:51 -0700 Subject: [PATCH 11/34] [SPARK-2485][SQL] Lock usage of hive client. Author: Michael Armbrust Closes #1412 from marmbrus/lockHiveClient and squashes the following commits: 4bc9d5a [Michael Armbrust] protected[hive] 22e9177 [Michael Armbrust] Add comments. 7aa8554 [Michael Armbrust] Don't lock on hive's object. a6edc5f [Michael Armbrust] Lock usage of hive client. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f83068860701f..8db60d32767b5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -43,14 +43,15 @@ import scala.collection.JavaConversions._ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ - val client = Hive.get(hive.hiveconf) + /** Connection to hive metastore. Usages should lock on `this`. */ + protected[hive] val client = Hive.get(hive.hiveconf) val caseSensitive: Boolean = false def lookupRelation( db: Option[String], tableName: String, - alias: Option[String]): LogicalPlan = { + alias: Option[String]): LogicalPlan = synchronized { val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) val table = client.getTable(databaseName, tblName) From 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Tue, 15 Jul 2014 00:22:06 -0700 Subject: [PATCH 12/34] discarded exceeded completedDrivers When completedDrivers number exceeds the threshold, the first Max(spark.deploy.retainedDrivers, 1) will be discarded. Author: lianhuiwang Closes #1114 from lianhuiwang/retained-drivers and squashes the following commits: 8789418 [lianhuiwang] discarded exceeded completedDrivers --- .../main/scala/org/apache/spark/deploy/master/Master.scala | 5 +++++ 1 file changed, 5 insertions(+) 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 d9f8105992a10..9fa556d522ba7 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 @@ -57,6 +57,7 @@ private[spark] class Master( def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") @@ -741,6 +742,10 @@ private[spark] class Master( case Some(driver) => logInfo(s"Removing driver: $driverId") drivers -= driver + if (completedDrivers.size >= RETAINED_DRIVERS) { + val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + completedDrivers.trimStart(toRemove) + } completedDrivers += driver persistenceEngine.removeDriver(driver) driver.state = finalState From dd95abada78b4d0aec97dacda50fdfd74464b073 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 15 Jul 2014 01:46:57 -0700 Subject: [PATCH 13/34] [SPARK-2399] Add support for LZ4 compression. Based on Greg Bowyer's patch from JIRA https://issues.apache.org/jira/browse/SPARK-2399 Author: Reynold Xin Closes #1416 from rxin/lz4 and squashes the following commits: 6c8fefe [Reynold Xin] Fixed typo. 8a14d38 [Reynold Xin] [SPARK-2399] Add support for LZ4 compression. --- core/pom.xml | 4 ++++ .../apache/spark/io/CompressionCodec.scala | 22 +++++++++++++++++++ .../spark/io/CompressionCodecSuite.scala | 6 +++++ docs/configuration.md | 10 ++++++++- pom.xml | 5 +++++ 5 files changed, 46 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 4ed920a750fff..1054cec4d77bb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -114,6 +114,10 @@ org.xerial.snappy snappy-java + + net.jpountz.lz4 + lz4 + com.twitter chill_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 4b0fe1ab82999..33402c927c732 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -20,6 +20,7 @@ package org.apache.spark.io import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} +import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf @@ -59,6 +60,27 @@ private[spark] object CompressionCodec { } +/** + * :: DeveloperApi :: + * LZ4 implementation of [[org.apache.spark.io.CompressionCodec]]. + * Block size can be configured by `spark.io.compression.lz4.block.size`. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. + */ +@DeveloperApi +class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { + + override def compressedOutputStream(s: OutputStream): OutputStream = { + val blockSize = conf.getInt("spark.io.compression.lz4.block.size", 32768) + new LZ4BlockOutputStream(s, blockSize) + } + + override def compressedInputStream(s: InputStream): InputStream = new LZ4BlockInputStream(s) +} + + /** * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 68a0ea36aa545..42fc395fa698d 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -50,6 +50,12 @@ class CompressionCodecSuite extends FunSuite { testCodec(codec) } + test("lz4 compression codec") { + val codec = CompressionCodec.createCodec(conf, classOf[LZ4CompressionCodec].getName) + assert(codec.getClass === classOf[LZ4CompressionCodec]) + testCodec(codec) + } + test("lzf compression codec") { val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName) assert(codec.getClass === classOf[LZFCompressionCodec]) diff --git a/docs/configuration.md b/docs/configuration.md index 07aa4c035446b..19fd980e6088f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -350,7 +350,15 @@ Apart from these, the following properties are also available, and may be useful 32768 Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec - is used. + is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. + + + + spark.io.compression.lz4.block.size + 32768 + + Block size (in bytes) used in LZ4 compression, in the case when LZ4 compression codec + is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used. diff --git a/pom.xml b/pom.xml index fa80707d0929c..d570f3e6b9321 100644 --- a/pom.xml +++ b/pom.xml @@ -297,6 +297,11 @@ snappy-java 1.0.5 + + net.jpountz.lz4 + lz4 + 1.2.0 + com.clearspring.analytics stream From 52beb20f7904e0333198b9b14619366ddf53ab85 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 15 Jul 2014 02:14:58 -0700 Subject: [PATCH 14/34] [SPARK-2477][MLlib] Using appendBias for adding intercept in GeneralizedLinearAlgorithm Instead of using prependOne currently in GeneralizedLinearAlgorithm, we would like to use appendBias for 1) keeping the indices of original training set unchanged by adding the intercept into the last element of vector and 2) using the same public API for consistently adding intercept. Author: DB Tsai Closes #1410 from dbtsai/SPARK-2477_intercept_with_appendBias and squashes the following commits: 011432c [DB Tsai] From Alpine Data Labs --- .../GeneralizedLinearAlgorithm.scala | 21 +++++-------------- 1 file changed, 5 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 8cca926f1c92e..fe41863bce985 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,13 +17,12 @@ package org.apache.spark.mllib.regression -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.util.MLUtils._ /** * :: DeveloperApi :: @@ -124,16 +123,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] run(input, initialWeights) } - /** Prepends one to the input vector. */ - private def prependOne(vector: Vector): Vector = { - val vector1 = vector.toBreeze match { - case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv) - case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) - case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) - } - Vectors.fromBreeze(vector1) - } - /** * Run the algorithm with the configured parameters on an input RDD * of LabeledPoint entries starting from the initial weights provided. @@ -147,23 +136,23 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features))) + input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features))) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - prependOne(initialWeights) + appendBias(initialWeights) } else { initialWeights } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0 + val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0 val weights = if (addIntercept) { - Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size)) + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) } else { weightsWithIntercept } From 8f1d4226c285e33d2fb839d3163bb374eb6db0e7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 15 Jul 2014 02:15:29 -0700 Subject: [PATCH 15/34] Update README.md to include a slightly more informative project description. (cherry picked from commit 401083be9f010f95110a819a49837ecae7d9c4ec) Signed-off-by: Reynold Xin --- README.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index f6e7f51091314..096f13e716368 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,13 @@ # Apache Spark -Lightning-Fast Cluster Computing - +Spark is a fast and general cluster computing system. It provides +high-level APIs in Scala, Java, and Python, and an optimized engine that +supports general computation graphs for data analysis. It also supports a +rich set of higher-level tools including Spark SQL for SQL and structured +data processing, MLLib for machine learning, GraphX for graph processing, +and Spark Streaming. + + ## Online Documentation From 6555618c8f39b4e7da9402c3fd9da7a75bf7794e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 15 Jul 2014 02:20:01 -0700 Subject: [PATCH 16/34] README update: added "for Big Data". --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 096f13e716368..f87e07aa5cc90 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,6 @@ # Apache Spark -Spark is a fast and general cluster computing system. It provides +Spark is a fast and general cluster computing system for Big Data. It provides high-level APIs in Scala, Java, and Python, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and structured From 04b01bb101eeaf76c2e7c94c291669f0b2372c9a Mon Sep 17 00:00:00 2001 From: Alexander Ulanov Date: Tue, 15 Jul 2014 08:40:22 -0700 Subject: [PATCH 17/34] [MLLIB] [SPARK-2222] Add multiclass evaluation metrics Adding two classes: 1) MulticlassMetrics implements various multiclass evaluation metrics 2) MulticlassMetricsSuite implements unit tests for MulticlassMetrics Author: Alexander Ulanov Author: unknown Author: Xiangrui Meng Closes #1155 from avulanov/master and squashes the following commits: 2eae80f [Alexander Ulanov] Merge pull request #1 from mengxr/avulanov-master 5ebeb08 [Xiangrui Meng] minor updates 79c3555 [Alexander Ulanov] Addressing reviewers comments mengxr 0fa9511 [Alexander Ulanov] Addressing reviewers comments mengxr f0dadc9 [Alexander Ulanov] Addressing reviewers comments mengxr 4811378 [Alexander Ulanov] Removing println 87fb11f [Alexander Ulanov] Addressing reviewers comments mengxr. Added confusion matrix e3db569 [Alexander Ulanov] Addressing reviewers comments mengxr. Added true positive rate and false positive rate. Test suite code style. a7e8bf0 [Alexander Ulanov] Addressing reviewers comments mengxr c3a77ad [Alexander Ulanov] Addressing reviewers comments mengxr e2c91c3 [Alexander Ulanov] Fixes to mutliclass metics d5ce981 [unknown] Comments about Double a5c8ba4 [unknown] Unit tests. Class rename fcee82d [unknown] Unit tests. Class rename d535d62 [unknown] Multiclass evaluation --- .../mllib/evaluation/MulticlassMetrics.scala | 190 ++++++++++++++++++ .../evaluation/MulticlassMetricsSuite.scala | 90 +++++++++ 2 files changed, 280 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala new file mode 100644 index 0000000000000..666362ae6739a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import scala.collection.Map + +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Matrices, Matrix} +import org.apache.spark.rdd.RDD + +/** + * ::Experimental:: + * Evaluator for multiclass classification. + * + * @param predictionAndLabels an RDD of (prediction, label) pairs. + */ +@Experimental +class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { + + private lazy val labelCountByClass: Map[Double, Long] = predictionAndLabels.values.countByValue() + private lazy val labelCount: Long = labelCountByClass.values.sum + private lazy val tpByClass: Map[Double, Int] = predictionAndLabels + .map { case (prediction, label) => + (label, if (label == prediction) 1 else 0) + }.reduceByKey(_ + _) + .collectAsMap() + private lazy val fpByClass: Map[Double, Int] = predictionAndLabels + .map { case (prediction, label) => + (prediction, if (prediction != label) 1 else 0) + }.reduceByKey(_ + _) + .collectAsMap() + private lazy val confusions = predictionAndLabels + .map { case (prediction, label) => + ((label, prediction), 1) + }.reduceByKey(_ + _) + .collectAsMap() + + /** + * Returns confusion matrix: + * predicted classes are in columns, + * they are ordered by class label ascending, + * as in "labels" + */ + def confusionMatrix: Matrix = { + val n = labels.size + val values = Array.ofDim[Double](n * n) + var i = 0 + while (i < n) { + var j = 0 + while (j < n) { + values(i + j * n) = confusions.getOrElse((labels(i), labels(j)), 0).toDouble + j += 1 + } + i += 1 + } + Matrices.dense(n, n, values) + } + + /** + * Returns true positive rate for a given label (category) + * @param label the label. + */ + def truePositiveRate(label: Double): Double = recall(label) + + /** + * Returns false positive rate for a given label (category) + * @param label the label. + */ + def falsePositiveRate(label: Double): Double = { + val fp = fpByClass.getOrElse(label, 0) + fp.toDouble / (labelCount - labelCountByClass(label)) + } + + /** + * Returns precision for a given label (category) + * @param label the label. + */ + def precision(label: Double): Double = { + val tp = tpByClass(label) + val fp = fpByClass.getOrElse(label, 0) + if (tp + fp == 0) 0 else tp.toDouble / (tp + fp) + } + + /** + * Returns recall for a given label (category) + * @param label the label. + */ + def recall(label: Double): Double = tpByClass(label).toDouble / labelCountByClass(label) + + /** + * Returns f-measure for a given label (category) + * @param label the label. + * @param beta the beta parameter. + */ + def fMeasure(label: Double, beta: Double): Double = { + val p = precision(label) + val r = recall(label) + val betaSqrd = beta * beta + if (p + r == 0) 0 else (1 + betaSqrd) * p * r / (betaSqrd * p + r) + } + + /** + * Returns f1-measure for a given label (category) + * @param label the label. + */ + def fMeasure(label: Double): Double = fMeasure(label, 1.0) + + /** + * Returns precision + */ + lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + + /** + * Returns recall + * (equals to precision for multiclass classifier + * because sum of all false positives is equal to sum + * of all false negatives) + */ + lazy val recall: Double = precision + + /** + * Returns f-measure + * (equals to precision and recall because precision equals recall) + */ + lazy val fMeasure: Double = precision + + /** + * Returns weighted true positive rate + * (equals to precision, recall and f-measure) + */ + lazy val weightedTruePositiveRate: Double = weightedRecall + + /** + * Returns weighted false positive rate + */ + lazy val weightedFalsePositiveRate: Double = labelCountByClass.map { case (category, count) => + falsePositiveRate(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged recall + * (equals to precision, recall and f-measure) + */ + lazy val weightedRecall: Double = labelCountByClass.map { case (category, count) => + recall(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged precision + */ + lazy val weightedPrecision: Double = labelCountByClass.map { case (category, count) => + precision(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged f-measure + * @param beta the beta parameter. + */ + def weightedFMeasure(beta: Double): Double = labelCountByClass.map { case (category, count) => + fMeasure(category, beta) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged f1-measure + */ + lazy val weightedFMeasure: Double = labelCountByClass.map { case (category, count) => + fMeasure(category, 1.0) * count.toDouble / labelCount + }.sum + + /** + * Returns the sequence of labels in ascending order + */ + lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala new file mode 100644 index 0000000000000..1ea503971c864 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Matrices +import org.apache.spark.mllib.util.LocalSparkContext + +class MulticlassMetricsSuite extends FunSuite with LocalSparkContext { + test("Multiclass evaluation metrics") { + /* + * Confusion matrix for 3-class classification with total 9 instances: + * |2|1|1| true class0 (4 instances) + * |1|3|0| true class1 (4 instances) + * |0|0|1| true class2 (1 instance) + */ + val confusionMatrix = Matrices.dense(3, 3, Array(2, 1, 0, 1, 3, 0, 1, 0, 1)) + val labels = Array(0.0, 1.0, 2.0) + val predictionAndLabels = sc.parallelize( + Seq((0.0, 0.0), (0.0, 1.0), (0.0, 0.0), (1.0, 0.0), (1.0, 1.0), + (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) + val metrics = new MulticlassMetrics(predictionAndLabels) + val delta = 0.0000001 + val fpRate0 = 1.0 / (9 - 4) + val fpRate1 = 1.0 / (9 - 4) + val fpRate2 = 1.0 / (9 - 1) + val precision0 = 2.0 / (2 + 1) + val precision1 = 3.0 / (3 + 1) + val precision2 = 1.0 / (1 + 1) + val recall0 = 2.0 / (2 + 2) + val recall1 = 3.0 / (3 + 1) + val recall2 = 1.0 / (1 + 0) + val f1measure0 = 2 * precision0 * recall0 / (precision0 + recall0) + val f1measure1 = 2 * precision1 * recall1 / (precision1 + recall1) + val f1measure2 = 2 * precision2 * recall2 / (precision2 + recall2) + val f2measure0 = (1 + 2 * 2) * precision0 * recall0 / (2 * 2 * precision0 + recall0) + val f2measure1 = (1 + 2 * 2) * precision1 * recall1 / (2 * 2 * precision1 + recall1) + val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) + + assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) + assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) + assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) + assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) + assert(math.abs(metrics.precision(0.0) - precision0) < delta) + assert(math.abs(metrics.precision(1.0) - precision1) < delta) + assert(math.abs(metrics.precision(2.0) - precision2) < delta) + assert(math.abs(metrics.recall(0.0) - recall0) < delta) + assert(math.abs(metrics.recall(1.0) - recall1) < delta) + assert(math.abs(metrics.recall(2.0) - recall2) < delta) + assert(math.abs(metrics.fMeasure(0.0) - f1measure0) < delta) + assert(math.abs(metrics.fMeasure(1.0) - f1measure1) < delta) + assert(math.abs(metrics.fMeasure(2.0) - f1measure2) < delta) + assert(math.abs(metrics.fMeasure(0.0, 2.0) - f2measure0) < delta) + assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) + assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) + + assert(math.abs(metrics.recall - + (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) + assert(math.abs(metrics.recall - metrics.precision) < delta) + assert(math.abs(metrics.recall - metrics.fMeasure) < delta) + assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.weightedFalsePositiveRate - + ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) + assert(math.abs(metrics.weightedPrecision - + ((4.0 / 9) * precision0 + (4.0 / 9) * precision1 + (1.0 / 9) * precision2)) < delta) + assert(math.abs(metrics.weightedRecall - + ((4.0 / 9) * recall0 + (4.0 / 9) * recall1 + (1.0 / 9) * recall2)) < delta) + assert(math.abs(metrics.weightedFMeasure - + ((4.0 / 9) * f1measure0 + (4.0 / 9) * f1measure1 + (1.0 / 9) * f1measure2)) < delta) + assert(math.abs(metrics.weightedFMeasure(2.0) - + ((4.0 / 9) * f2measure0 + (4.0 / 9) * f2measure1 + (1.0 / 9) * f2measure2)) < delta) + assert(metrics.labels.sameElements(labels)) + } +} From cb09e93c1d7ef9c8f0a1abe4e659783c74993a4e Mon Sep 17 00:00:00 2001 From: William Benton Date: Tue, 15 Jul 2014 09:13:39 -0700 Subject: [PATCH 18/34] Reformat multi-line closure argument. Author: William Benton Closes #1419 from willb/reformat-2486 and squashes the following commits: 2676231 [William Benton] Reformat multi-line closure argument. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 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 d72c97bbe816a..10c33d67e7683 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -809,11 +809,12 @@ private[spark] object Utils extends Logging { */ def getCallSite: CallSite = { val trace = Thread.currentThread.getStackTrace() - .filterNot((ste:StackTraceElement) => + .filterNot { ste:StackTraceElement => // When running under some profilers, the current stack trace might contain some bogus // frames. This is intended to ensure that we don't crash in these situations by // ignoring any frames that we can't examine. - (ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace"))) + (ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace")) + } // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD From 9dd635eb5df52835b3b7f4f2b9c789da9e813c71 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 15 Jul 2014 10:46:17 -0700 Subject: [PATCH 19/34] SPARK-2480: Resolve sbt warnings "NOTE: SPARK_YARN is deprecated, please use -Pyarn flag" Author: witgo Closes #1404 from witgo/run-tests and squashes the following commits: f703aee [witgo] fix Note: implicit method fromPairDStream is not applicable here because it comes after the application point and it lacks an explicit result type 2944f51 [witgo] Remove "NOTE: SPARK_YARN is deprecated, please use -Pyarn flag" ef59c70 [witgo] fix Note: implicit method fromPairDStream is not applicable here because it comes after the application point and it lacks an explicit result type 6cefee5 [witgo] Remove "NOTE: SPARK_YARN is deprecated, please use -Pyarn flag" --- dev/run-tests | 7 +++---- dev/scalastyle | 6 +++--- docs/hadoop-third-party-distributions.md | 4 ++-- docs/sql-programming-guide.md | 2 +- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index edd17b53b3d8c..51e4def0f835a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,8 +21,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd $FWDIR -export SPARK_HADOOP_VERSION=2.3.0 -export SPARK_YARN=true +export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" # Remove work directory rm -rf ./work @@ -66,8 +65,8 @@ echo "=========================================================================" # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. if [ -n "$_RUN_SQL_TESTS" ]; then - echo -e "q\n" | SPARK_HIVE=true sbt/sbt clean package assembly/assembly test | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive" sbt/sbt clean package \ + assembly/assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" else echo -e "q\n" | sbt/sbt clean package assembly/assembly test | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" diff --git a/dev/scalastyle b/dev/scalastyle index 0e8fd5cc8d64c..a02d06912f238 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,12 +17,12 @@ # limitations under the License. # -echo -e "q\n" | SPARK_HIVE=true sbt/sbt scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | SPARK_HADOOP_VERSION=0.23.9 SPARK_YARN=true sbt/sbt yarn-alpha/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt yarn/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | grep -e "\") diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 32403bc6957a2..ab1023b8f1842 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -48,9 +48,9 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. -In SBT, the equivalent can be achieved by setting the SPARK_HADOOP_VERSION flag: +In SBT, the equivalent can be achieved by setting the the `hadoop.version` property: - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + sbt/sbt -Dhadoop.version=1.0.4 assembly # Linking Applications to the Hadoop Version diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 522c83884ef42..38728534a46e0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -474,7 +474,7 @@ anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`' (or use `-Phive` for maven). +In order to use Hive you must first run '`sbt/sbt -Phive assembly/assembly`' (or use `-Phive` for maven). This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. From 72ea56da8e383c61c6f18eeefef03b9af00f5158 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 15 Jul 2014 13:52:56 -0500 Subject: [PATCH 20/34] SPARK-1291: Link the spark UI to RM ui in yarn-client mode Author: witgo Closes #1112 from witgo/SPARK-1291 and squashes the following commits: 6022bcd [witgo] review commit 1fbb925 [witgo] add addAmIpFilter to yarn alpha 210299c [witgo] review commit 1b92a07 [witgo] review commit 6896586 [witgo] Add comments to addWebUIFilter 3e9630b [witgo] review commit 142ee29 [witgo] review commit 1fe7710 [witgo] Link the spark UI to RM ui in yarn-client mode --- .../cluster/CoarseGrainedClusterMessage.scala | 3 +++ .../CoarseGrainedSchedulerBackend.scala | 18 +++++++++++++++ .../scala/org/apache/spark/ui/UIUtils.scala | 11 ++++++++- .../spark/deploy/yarn/ExecutorLauncher.scala | 22 +++++++++++++++--- .../cluster/YarnClientSchedulerBackend.scala | 1 + .../spark/deploy/yarn/ExecutorLauncher.scala | 23 ++++++++++++++++--- 6 files changed, 71 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 318e16552201c..6abf6d930c155 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -66,4 +66,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage + case class AddWebUIFilter(filterName:String, filterParams: String, proxyBase :String) + extends CoarseGrainedClusterMessage + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0f5545e2ed65f..9f085eef46720 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -31,6 +31,7 @@ import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.ui.JettyUtils /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -136,6 +137,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A removeExecutor(executorId, reason) sender ! true + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) + sender ! true case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) @@ -276,6 +280,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } false } + + // Add filters to the SparkUI + def addWebUIFilter(filterName: String, filterParams: String, proxyBase: String) { + if (proxyBase != null && proxyBase.nonEmpty) { + System.setProperty("spark.ui.proxyBase", proxyBase) + } + + if (Seq(filterName, filterParams).forall(t => t != null && t.nonEmpty)) { + logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") + conf.set("spark.ui.filters", filterName) + conf.set(s"spark.$filterName.params", filterParams) + JettyUtils.addFilters(scheduler.sc.ui.getHandlers, conf) + } + } } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 9cb50d9b83dda..e07aa2ee3a5a2 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -136,7 +136,16 @@ private[spark] object UIUtils extends Logging { } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") + def uiRoot: String = { + if (System.getenv("APPLICATION_WEB_PROXY_BASE") != null) { + System.getenv("APPLICATION_WEB_PROXY_BASE") + } else if (System.getProperty("spark.ui.proxyBase") != null) { + System.getProperty("spark.ui.proxyBase") + } + else { + "" + } + } def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index bfdb6232f5113..a86ad256dfa39 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -32,6 +32,7 @@ import akka.actor.Terminated import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.scheduler.SplitInfo import org.apache.spark.deploy.SparkHadoopUtil @@ -81,6 +82,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") driverClosed = true + case x: AddWebUIFilter => + logInfo(s"Add WebUI Filter. $x") + driver ! x } } @@ -111,7 +115,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } waitForSparkMaster() - + addAmIpFilter() // Allocate all containers allocateExecutors() @@ -171,7 +175,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") + val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") + logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) .asInstanceOf[RegisterApplicationMasterRequest] appMasterRequest.setApplicationAttemptId(appAttemptId) @@ -180,10 +185,21 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) // What do we provide here ? Might make sense to expose something sensible later ? - appMasterRequest.setTrackingUrl("") + appMasterRequest.setTrackingUrl(appUIAddress) resourceManager.registerApplicationMaster(appMasterRequest) } + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val proxy = YarnConfiguration.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + val uriBase = "http://" + proxy + proxyBase + val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) + } + private def waitForSparkMaster() { logInfo("Waiting for spark driver to be reachable.") var driverUp = false diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 0f9fdcfcb6510..1b37c4bb13f49 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -48,6 +48,7 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort + conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index f71ad036ce0f2..5ac95f3798723 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -31,10 +31,12 @@ import akka.actor.Terminated import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.yarn.webapp.util.WebAppUtils /** * An application master that allocates executors on behalf of a driver that is running outside @@ -82,6 +84,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") driverClosed = true + case x: AddWebUIFilter => + logInfo(s"Add WebUI Filter. $x") + driver ! x } } @@ -99,6 +104,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp registerApplicationMaster() waitForSparkMaster() + addAmIpFilter() // Allocate all containers allocateExecutors() @@ -142,9 +148,20 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") - // TODO: Find out client's Spark UI address and fill in here? - amClient.registerApplicationMaster(Utils.localHostName(), 0, "") + val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") + logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") + amClient.registerApplicationMaster(Utils.localHostName(), 0, appUIAddress) + } + + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + val uriBase = "http://" + proxy + proxyBase + val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) } private def waitForSparkMaster() { From e7ec815d9a2b0f89a56dc7dd3106c31a09492028 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 15 Jul 2014 13:13:33 -0700 Subject: [PATCH 21/34] Added LZ4 to compression codec in configuration page. Author: Reynold Xin Closes #1417 from rxin/lz4 and squashes the following commits: 472f6a1 [Reynold Xin] Set the proper default. 9cf0b2f [Reynold Xin] Added LZ4 to compression codec in configuration page. --- docs/configuration.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 19fd980e6088f..9d3fe7441486d 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -336,13 +336,12 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec - org.apache.spark.io.
LZFCompressionCodec + org.apache.spark.io.
SnappyCompressionCodec The codec used to compress internal data such as RDD partitions and shuffle outputs. - By default, Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec - and org.apache.spark.io.SnappyCompressionCodec. Of these two choices, - Snappy offers faster compression and decompression, while LZF offers a better compression - ratio. + By default, Spark provides three codecs: org.apache.spark.io.LZ4CompressionCodec, + org.apache.spark.io.LZFCompressionCodec, + and org.apache.spark.io.SnappyCompressionCodec. From a21f9a7543309320bb2791468243c8f10bc6e81b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 14:00:54 -0700 Subject: [PATCH 22/34] [SPARK-2471] remove runtime scope for jets3t The assembly jar (built by sbt) doesn't include jets3t if we set it to runtime only, but I don't know whether it was set this way for a particular reason. CC: srowen ScrapCodes Author: Xiangrui Meng Closes #1402 from mengxr/jets3t and squashes the following commits: bfa2d17 [Xiangrui Meng] remove runtime scope for jets3t --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index d570f3e6b9321..4e2d64a833640 100644 --- a/pom.xml +++ b/pom.xml @@ -614,7 +614,6 @@ net.java.dev.jets3t jets3t ${jets3t.version} - runtime commons-logging From 0f98ef1a2c9ecf328f6c5918808fa5ca486e8afd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 15 Jul 2014 14:01:48 -0700 Subject: [PATCH 23/34] [SPARK-2483][SQL] Fix parsing of repeated, nested data access. Author: Michael Armbrust Closes #1411 from marmbrus/nestedRepeated and squashes the following commits: 044fa09 [Michael Armbrust] Fix parsing of repeated, nested data access. --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 5 +---- .../spark/sql/hive/execution/HiveResolutionSuite.scala | 10 ++++++++-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b70104dd5be5a..56aa27a208828 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -870,10 +870,7 @@ private[hive] object HiveQl { nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) - // The precidence for . seems to be wrong, so [] binds tighter an we need to go inside to - // find the underlying attribute references. - case GetItem(UnresolvedAttribute(qualifierName), ordinal) => - GetItem(UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)), ordinal) + case other => GetField(other, attr) } /* Stars (*) */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 67594b57d3dfa..fb03db12a0b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -case class Data(a: Int, B: Int, n: Nested) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) /** @@ -53,12 +53,18 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection - TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil) + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerAsTable("caseSensitivityTest") hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") } + test("nested repeated resolution") { + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerAsTable("nestedRepeatedTest") + assert(hql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) + } + /** * Negative examples. Currently only left here for documentation purposes. * TODO(marmbrus): Test that catalyst fails on these queries. From bcd0c30c7eea4c50301cb732c733fdf4d4142060 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 15 Jul 2014 14:04:01 -0700 Subject: [PATCH 24/34] [SQL] Whitelist more Hive tests. Author: Michael Armbrust Closes #1396 from marmbrus/moreTests and squashes the following commits: 6660b60 [Michael Armbrust] Blacklist a test that requires DFS command. 8b6001c [Michael Armbrust] Add golden files. ccd8f97 [Michael Armbrust] Whitelist more tests. --- ...imizer1-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer1-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a | 1 + ...mizer1-11-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer1-12-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-13-f9f839aedb3a350719c0cbc53a06ace5 | 0 ...mizer1-14-dae4256e08d595317f8e09a56354a3d9 | 1 + ...mizer1-15-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-16-43f356b36962f2bade5706d8cf5ae6b4 | 0 ...mizer1-17-dae4256e08d595317f8e09a56354a3d9 | 1 + ...mizer1-18-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer1-19-b9d963d24994c47c3776dda6f7d3881f | 1 + ...imizer1-2-42a0eedc3751f792ad5438b2c64d3897 | 0 ...mizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 | 0 ...mizer1-21-16c57348be42ca3cc2f80f7f92265696 | 1 + ...mizer1-22-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...timizer1-23-2cdc77fd60449f3547cf95d8eb09a2 | 0 ...mizer1-24-16c57348be42ca3cc2f80f7f92265696 | 1 + ...mizer1-25-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-26-8bcdcc5f01508f576d7bd6422c939225 | 0 ...mizer1-27-d31433f229e853e8b8440b4ddc63c80e | 1 + ...mizer1-28-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-29-941ecfef9448ecff56cc16bcfb233ee4 | 0 ...imizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a | 1 + ...mizer1-30-d31433f229e853e8b8440b4ddc63c80e | 1 + ...mizer1-31-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-32-ef6502d6b282c8a6d228bba395b24724 | 0 ...mizer1-33-ea87e76dba02a46cb958148333e397b7 | 1 + ...mizer1-34-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-35-b79b220859c09354e23b533c105ccbab | 0 ...mizer1-36-ea87e76dba02a46cb958148333e397b7 | 1 + ...mizer1-37-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-38-638e5300f4c892c2bf27bd91a8f81b64 | 0 ...mizer1-39-66010469a9cdb66851da9a727ef9fdad | 1 + ...imizer1-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-40-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...timizer1-41-3514c74c7f68f2d70cc6d51ac46c20 | 0 ...mizer1-42-66010469a9cdb66851da9a727ef9fdad | 1 + ...mizer1-43-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-44-7490df6719cd7e47aa08dbcbc3266a92 | 0 ...mizer1-45-e71195e7d9f557e2abc7f03462d22dba | 1 + ...mizer1-46-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 | 0 ...mizer1-48-e71195e7d9f557e2abc7f03462d22dba | 1 + ...mizer1-49-b1e2ade89ae898650f0be4f796d8947b | 1 + ...imizer1-5-a1c80c68b9a7597096c2580c3766f7f7 | 0 ...mizer1-50-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-51-fcf9bcb522f542637ccdea863b408448 | 0 ...imizer1-52-3070366869308907e54797927805603 | 1 + ...mizer1-53-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 | 0 ...imizer1-55-3070366869308907e54797927805603 | 1 + ...mizer1-56-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 | 0 ...imizer1-58-a6bba6d9b422adb386b35c62cecb548 | 1 + ...mizer1-59-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a | 1 + ...mizer1-60-d6bbaf0d40010159095e4cac025c50c5 | 0 ...imizer1-61-a6bba6d9b422adb386b35c62cecb548 | 1 + ...imizer1-7-24ca942f094b14b92086305cc125e833 | 1 + ...imizer1-8-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...imizer1-9-d5bea91b4edb8be0428a336ff9c21dde | 0 ...mizer10-0-b1e2ade89ae898650f0be4f796d8947b | 1 + ...mizer10-1-b9d963d24994c47c3776dda6f7d3881f | 1 + ...izer10-10-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer10-11-a1b7af95dfd01783c07aa23208d6160 | 0 ...izer10-12-1322cff0bdf29aab32e638ad48c71ff9 | 5 ++++ ...izer10-13-b9d963d24994c47c3776dda6f7d3881f | 1 + ...izer10-14-934b668c11600dc9c013c2ddc4c0d68c | 0 ...mizer10-15-430ff20a144fb3dbf526232d9cb2baa | 23 +++++++++++++++++++ ...izer10-16-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...izer10-17-9d5521ecef1353d23fd2d4f7d78e7006 | 0 ...mizer10-18-430ff20a144fb3dbf526232d9cb2baa | 23 +++++++++++++++++++ ...imizer10-2-f9de06a4184ab1f42793327c1497437 | 0 ...mizer10-3-6a01aa7ca94cda4268af894b4fd852ea | 15 ++++++++++++ ...mizer10-4-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...mizer10-5-6f191930802d659058465b2e6de08dd3 | 0 ...mizer10-6-6a01aa7ca94cda4268af894b4fd852ea | 15 ++++++++++++ ...mizer10-7-b9d963d24994c47c3776dda6f7d3881f | 1 + ...mizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec | 0 ...mizer10-9-1322cff0bdf29aab32e638ad48c71ff9 | 5 ++++ ...pby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 | 0 ...pby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d | 0 ...pby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...egative-0-79b294d0081c3dfd36c5b8b5e78dc7fb | 1 + ...egative-1-f87339637a48bd1533493ebbed5432a7 | 0 ...egative-2-de7e5ac581b870fff10dc82c75c1c79e | 0 ...egative-3-be440c3f959ca53b758481aa90551984 | 0 ...egative-4-4dedc8057d76af264c198beaacd7f000 | 0 ...egative-5-543a20e69bd8987bc37a22c1c7ef33f1 | 0 ...egative-6-3f8274466914ad200b33a2c83fa6dab5 | 0 ...egative-7-fb7bf3783d4fb43673a202c4111d9092 | 0 ...tamp_3-10-7b1ec929239ee305ea9da46ebb990c67 | 1 + ...tamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a | 1 + ...tamp_3-12-165256158e3db1ce19c3c9db3c8011d2 | 0 ...stamp_3-3-6143888a940bfcac1133330764f5a31a | 0 ...stamp_3-4-935d0d2492beab99bbbba26ba62a1db4 | 1 + ...stamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 | 1 + ...stamp_3-6-6be5fe01c502cd24db32a3781c97a703 | 1 + ...stamp_3-7-6066ba0451cd0fcfac4bea6376e72add | 1 + ...stamp_3-8-22e03daa775eab145d39ec0730953f7e | 1 + ...stamp_3-9-ffc79abb874323e165963aa39f460a9b | 1 + ...mp_null-3-222c5ea127c747c71738b5dc5b80459c | 1 + ...mp_null-4-ffc86f5c714eceabc36e92931b96beb0 | 1 + .../execution/HiveCompatibilitySuite.scala | 16 +++++++++++++ 105 files changed, 163 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-13-f9f839aedb3a350719c0cbc53a06ace5 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-16-43f356b36962f2bade5706d8cf5ae6b4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-2-42a0eedc3751f792ad5438b2c64d3897 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-23-2cdc77fd60449f3547cf95d8eb09a2 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-29-941ecfef9448ecff56cc16bcfb233ee4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-5-a1c80c68b9a7597096c2580c3766f7f7 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-9-d5bea91b4edb8be0428a336ff9c21dde create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-11-a1b7af95dfd01783c07aa23208d6160 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-14-934b668c11600dc9c013c2ddc4c0d68c create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-17-9d5521ecef1353d23fd2d4f7d78e7006 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-2-f9de06a4184ab1f42793327c1497437 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-5-6f191930802d659058465b2e6de08dd3 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d create mode 100644 sql/hive/src/test/resources/golden/groupby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-1-f87339637a48bd1533493ebbed5432a7 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-2-de7e5ac581b870fff10dc82c75c1c79e create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-3-be440c3f959ca53b758481aa90551984 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-4-4dedc8057d76af264c198beaacd7f000 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-5-543a20e69bd8987bc37a22c1c7ef33f1 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-6-3f8274466914ad200b33a2c83fa6dab5 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b create mode 100644 sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c create mode 100644 sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-13-f9f839aedb3a350719c0cbc53a06ace5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-13-f9f839aedb3a350719c0cbc53a06ace5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 b/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 new file mode 100644 index 0000000000000..235736a2807b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 @@ -0,0 +1 @@ +3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-16-43f356b36962f2bade5706d8cf5ae6b4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-16-43f356b36962f2bade5706d8cf5ae6b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 b/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 new file mode 100644 index 0000000000000..235736a2807b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 @@ -0,0 +1 @@ +3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-2-42a0eedc3751f792ad5438b2c64d3897 b/sql/hive/src/test/resources/golden/correlationoptimizer1-2-42a0eedc3751f792ad5438b2c64d3897 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 b/sql/hive/src/test/resources/golden/correlationoptimizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 b/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-23-2cdc77fd60449f3547cf95d8eb09a2 b/sql/hive/src/test/resources/golden/correlationoptimizer1-23-2cdc77fd60449f3547cf95d8eb09a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 b/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 b/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e b/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-29-941ecfef9448ecff56cc16bcfb233ee4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-29-941ecfef9448ecff56cc16bcfb233ee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e b/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 b/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 new file mode 100644 index 0000000000000..5aa2d482094af --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 @@ -0,0 +1 @@ +79136 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab b/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 new file mode 100644 index 0000000000000..5aa2d482094af --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 @@ -0,0 +1 @@ +79136 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 b/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad new file mode 100644 index 0000000000000..b4a3a9d327f47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad @@ -0,0 +1 @@ +3556 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 b/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad new file mode 100644 index 0000000000000..b4a3a9d327f47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad @@ -0,0 +1 @@ +3556 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 b/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba new file mode 100644 index 0000000000000..bb564e0fd06eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba @@ -0,0 +1 @@ +3556 510 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 b/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba new file mode 100644 index 0000000000000..bb564e0fd06eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba @@ -0,0 +1 @@ +3556 510 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-5-a1c80c68b9a7597096c2580c3766f7f7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-5-a1c80c68b9a7597096c2580c3766f7f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 b/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 new file mode 100644 index 0000000000000..edd216f16b190 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 @@ -0,0 +1 @@ +3556 661329102 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 b/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 new file mode 100644 index 0000000000000..edd216f16b190 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 @@ -0,0 +1 @@ +3556 661329102 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 new file mode 100644 index 0000000000000..9fde6099b4f08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 @@ -0,0 +1 @@ +2835 29 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 new file mode 100644 index 0000000000000..9fde6099b4f08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 @@ -0,0 +1 @@ +2835 29 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-9-d5bea91b4edb8be0428a336ff9c21dde b/sql/hive/src/test/resources/golden/correlationoptimizer1-9-d5bea91b4edb8be0428a336ff9c21dde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-11-a1b7af95dfd01783c07aa23208d6160 b/sql/hive/src/test/resources/golden/correlationoptimizer10-11-a1b7af95dfd01783c07aa23208d6160 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 b/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 new file mode 100644 index 0000000000000..9ea431d9f5d18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 @@ -0,0 +1,5 @@ +66 val_66 +98 val_98 +128 +146 val_146 +150 val_150 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-14-934b668c11600dc9c013c2ddc4c0d68c b/sql/hive/src/test/resources/golden/correlationoptimizer10-14-934b668c11600dc9c013c2ddc4c0d68c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa b/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa new file mode 100644 index 0000000000000..5abecc5df25fd --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa @@ -0,0 +1,23 @@ +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-17-9d5521ecef1353d23fd2d4f7d78e7006 b/sql/hive/src/test/resources/golden/correlationoptimizer10-17-9d5521ecef1353d23fd2d4f7d78e7006 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa b/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa new file mode 100644 index 0000000000000..5abecc5df25fd --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa @@ -0,0 +1,23 @@ +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-2-f9de06a4184ab1f42793327c1497437 b/sql/hive/src/test/resources/golden/correlationoptimizer10-2-f9de06a4184ab1f42793327c1497437 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea b/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea new file mode 100644 index 0000000000000..d00aeb4be0340 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-5-6f191930802d659058465b2e6de08dd3 b/sql/hive/src/test/resources/golden/correlationoptimizer10-5-6f191930802d659058465b2e6de08dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea b/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea new file mode 100644 index 0000000000000..d00aeb4be0340 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec b/sql/hive/src/test/resources/golden/correlationoptimizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 b/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 new file mode 100644 index 0000000000000..9ea431d9f5d18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 @@ -0,0 +1,5 @@ +66 val_66 +98 val_98 +128 +146 val_146 +150 val_150 diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 b/sql/hive/src/test/resources/golden/groupby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d b/sql/hive/src/test/resources/golden/groupby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/groupby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-1-f87339637a48bd1533493ebbed5432a7 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-1-f87339637a48bd1533493ebbed5432a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-2-de7e5ac581b870fff10dc82c75c1c79e b/sql/hive/src/test/resources/golden/limit_pushdown_negative-2-de7e5ac581b870fff10dc82c75c1c79e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-3-be440c3f959ca53b758481aa90551984 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-3-be440c3f959ca53b758481aa90551984 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-4-4dedc8057d76af264c198beaacd7f000 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-4-4dedc8057d76af264c198beaacd7f000 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-5-543a20e69bd8987bc37a22c1c7ef33f1 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-5-543a20e69bd8987bc37a22c1c7ef33f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-6-3f8274466914ad200b33a2c83fa6dab5 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-6-3f8274466914ad200b33a2c83fa6dab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 b/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 new file mode 100644 index 0000000000000..1b0a140b5a384 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 @@ -0,0 +1 @@ +1.3041352164485E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a b/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a new file mode 100644 index 0000000000000..d7ff6cd63d9f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a @@ -0,0 +1 @@ +2011-04-29 20:46:56.4485 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a b/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 b/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 b/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 new file mode 100644 index 0000000000000..21e72e8ac3d7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 @@ -0,0 +1 @@ +48 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 b/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 new file mode 100644 index 0000000000000..ee3be2941da6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 @@ -0,0 +1 @@ +-31184 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add b/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add new file mode 100644 index 0000000000000..1cf1952ac0372 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add @@ -0,0 +1 @@ +1304135216 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e b/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e new file mode 100644 index 0000000000000..1cf1952ac0372 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e @@ -0,0 +1 @@ +1304135216 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b b/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b new file mode 100644 index 0000000000000..d21deca762237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b @@ -0,0 +1 @@ +1.30413517E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c b/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 b/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index cdfc2d0c17384..63dbe57c4c772 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -84,6 +84,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_java_method", "create_merge_compressed", + // DFS commands + "symlink_text_input_format", + // Weird DDL differences result in failures on jenkins. "create_like2", "create_view_translate", @@ -278,7 +281,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "compute_stats_string", "compute_stats_table", "convert_enum_to_string", + "correlationoptimizer1", + "correlationoptimizer10", "correlationoptimizer11", + "correlationoptimizer14", "correlationoptimizer15", "correlationoptimizer2", "correlationoptimizer3", @@ -296,6 +302,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ct_case_insensitive", "database_location", "database_properties", + "decimal_1", "decimal_4", "decimal_join", "default_partition_name", @@ -304,6 +311,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_formatted_view_partitioned", "diff_part_input_formats", "disable_file_format_check", + "disallow_incompatible_type_change_off", "drop_function", "drop_index", "drop_multi_partitions", @@ -359,8 +367,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_map_ppr", "groupby_multi_insert_common_distinct", "groupby_multi_single_reducer2", + "groupby_multi_single_reducer3", "groupby_mutli_insert_common_distinct", "groupby_neg_float", + "groupby_ppd", "groupby_ppr", "groupby_sort_10", "groupby_sort_2", @@ -400,6 +410,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input4", "input40", "input41", + "input49", "input4_cb_delim", "input6", "input7", @@ -491,6 +502,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "lateral_view_ppd", "leftsemijoin", "leftsemijoin_mr", + "limit_pushdown_negative", "lineage1", "literal_double", "literal_ints", @@ -598,6 +610,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "reduce_deduplicate", "reduce_deduplicate_exclude_gby", "reduce_deduplicate_exclude_join", + "reduce_deduplicate_extended", "reducesink_dedup", "rename_column", "router_join_ppr", @@ -646,7 +659,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats_publisher_error_1", "subq2", "tablename_with_select", + "timestamp_3", "timestamp_comparison", + "timestamp_null", + "timestamp_udf", "touch", "transform_ppr1", "transform_ppr2", From 8af46d58464b96471825ce376c3e11c8b1108c0e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 15 Jul 2014 14:06:45 -0700 Subject: [PATCH 25/34] [SPARK-2474][SQL] For a registered table in OverrideCatalog, the Analyzer failed to resolve references in the format of "tableName.fieldName" Please refer to JIRA (https://issues.apache.org/jira/browse/SPARK-2474) for how to reproduce the problem and my understanding of the root cause. Author: Yin Huai Closes #1406 from yhuai/SPARK-2474 and squashes the following commits: 96b1627 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2474 af36d65 [Yin Huai] Fix comment. be86ba9 [Yin Huai] Correct SQL console settings. c43ad00 [Yin Huai] Wrap the relation in a Subquery named by the table name in OverrideCatalog.lookupRelation. a5c2145 [Yin Huai] Support sql/console. --- project/SparkBuild.scala | 24 +++++++++++++++++++ .../spark/sql/catalyst/analysis/Catalog.scala | 3 ++- 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 754d54e89361f..5461d25d72d7e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -166,6 +166,9 @@ object SparkBuild extends PomBuild { /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) + /* Spark SQL Core console settings */ + enable(SQL.settings)(sql) + /* Hive console settings */ enable(Hive.settings)(hive) @@ -179,6 +182,27 @@ object SparkBuild extends PomBuild { } +object SQL { + + lazy val settings = Seq( + + initialCommands in console := + """ + |import org.apache.spark.sql.catalyst.analysis._ + |import org.apache.spark.sql.catalyst.dsl._ + |import org.apache.spark.sql.catalyst.errors._ + |import org.apache.spark.sql.catalyst.expressions._ + |import org.apache.spark.sql.catalyst.plans.logical._ + |import org.apache.spark.sql.catalyst.rules._ + |import org.apache.spark.sql.catalyst.types._ + |import org.apache.spark.sql.catalyst.util._ + |import org.apache.spark.sql.execution + |import org.apache.spark.sql.test.TestSQLContext._ + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin + ) + +} + object Hive { lazy val settings = Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 0d05d9808b407..616f1e2ecb60f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -113,11 +113,12 @@ trait OverrideCatalog extends Catalog { alias: Option[String] = None): LogicalPlan = { val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) val overriddenTable = overrides.get((dbName, tblName)) + val tableWithQualifers = overriddenTable.map(r => Subquery(tblName, r)) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. val withAlias = - overriddenTable.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) + tableWithQualifers.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) withAlias.getOrElse(super.lookupRelation(dbName, tblName, alias)) } From 61de65bc69f9a5fc396b76713193c6415436d452 Mon Sep 17 00:00:00 2001 From: William Benton Date: Tue, 15 Jul 2014 14:11:57 -0700 Subject: [PATCH 26/34] SPARK-2407: Added internal implementation of SQL SUBSTR() This replaces the Hive UDF for SUBSTR(ING) with an implementation in Catalyst and adds tests to verify correct operation. Author: William Benton Closes #1359 from willb/internalSqlSubstring and squashes the following commits: ccedc47 [William Benton] Fixed too-long line. a30a037 [William Benton] replace view bounds with implicit parameters ec35c80 [William Benton] Adds fixes from review: 4f3bfdb [William Benton] Added internal implementation of SQL SUBSTR() --- .../expressions/stringOperations.scala | 77 ++++++++++++++++++- .../ExpressionEvaluationSuite.scala | 49 ++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 5 ++ 3 files changed, 128 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index b3850533c3736..4bd7bf5a0cd8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,9 +19,11 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import org.apache.spark.sql.catalyst.types.DataType -import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.catalyst.types.BooleanType +import scala.collection.IndexedSeqOptimized + + +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types.{BinaryType, BooleanType, DataType, StringType} trait StringRegexExpression { self: BinaryExpression => @@ -205,3 +207,72 @@ case class EndsWith(left: Expression, right: Expression) extends BinaryExpression with StringComparison { def compare(l: String, r: String) = l.endsWith(r) } + +/** + * A function that takes a substring of its first argument starting at a given position. + * Defined for String and Binary types. + */ +case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression { + + type EvaluatedType = Any + + def nullable: Boolean = true + def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved") + } + if (str.dataType == BinaryType) str.dataType else StringType + } + + def references = children.flatMap(_.references).toSet + + override def children = str :: pos :: len :: Nil + + @inline + def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) + (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { + val len = str.length + // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and + // negative indices for start positions. If a start index i is greater than 0, it + // refers to element i-1 in the sequence. If a start index i is less than 0, it refers + // to the -ith element before the end of the sequence. If a start index i is 0, it + // refers to the first element. + + val start = startPos match { + case pos if pos > 0 => pos - 1 + case neg if neg < 0 => len + neg + case _ => 0 + } + + val end = sliceLen match { + case max if max == Integer.MAX_VALUE => max + case x => start + x + } + + str.slice(start, end) + } + + override def eval(input: Row): Any = { + val string = str.eval(input) + + val po = pos.eval(input) + val ln = len.eval(input) + + if ((string == null) || (po == null) || (ln == null)) { + null + } else { + val start = po.asInstanceOf[Int] + val length = ln.asInstanceOf[Int] + + string match { + case ba: Array[Byte] => slice(ba, start, length) + case other => slice(other.toString, start, length) + } + } + } + + override def toString = len match { + case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)" + case _ => s"SUBSTR($str, $pos, $len)" + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 84d72814778ba..f1d7aedcc2d2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -466,5 +466,54 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 === c2, false, row) checkEvaluation(c1 !== c2, true, row) } + + test("Substring") { + val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + + val s = 'a.string.at(0) + + // substring from zero position with less-than-full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row) + + // substring from zero position with full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(7, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(7, IntegerType)), "example", row) + + // substring from zero position with greater-than-full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(100, IntegerType)), "example", row) + + // substring from nonzero position with less-than-full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(2, IntegerType)), "xa", row) + + // substring from nonzero position with full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(6, IntegerType)), "xample", row) + + // substring from nonzero position with greater-than-full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(100, IntegerType)), "xample", row) + + // zero-length substring (within string bounds) + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(0, IntegerType)), "", row) + + // zero-length substring (beyond string bounds) + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), "", row) + + // substring(null, _, _) -> null + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null))) + + // substring(_, null, _) -> null + checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row) + + // substring(_, _, null) -> null + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(null, IntegerType)), null, row) + + // 2-arg substring from zero position + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + + // 2-arg substring from nonzero position + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 56aa27a208828..300e249f5b2e1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -860,6 +860,7 @@ private[hive] object HiveQl { val BETWEEN = "(?i)BETWEEN".r val WHEN = "(?i)WHEN".r val CASE = "(?i)CASE".r + val SUBSTR = "(?i)SUBSTR(?:ING)?".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -984,6 +985,10 @@ private[hive] object HiveQl { /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => + Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => + Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => From 502f90782ad474e2630ed5be4d3c4be7dab09c34 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 15 Jul 2014 17:56:17 -0700 Subject: [PATCH 27/34] [SQL] Attribute equality comparisons should be done by exprId. Author: Michael Armbrust Closes #1414 from marmbrus/exprIdResolution and squashes the following commits: 97b47bc [Michael Armbrust] Attribute equality comparisons should be done by exprId. --- .../spark/sql/columnar/InMemoryColumnarTableScan.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index ff7f664d8b529..88901debbb4e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -96,7 +96,11 @@ private[sql] case class InMemoryColumnarTableScan( new Iterator[Row] { // Find the ordinals of the requested columns. If none are requested, use the first. val requestedColumns = - if (attributes.isEmpty) Seq(0) else attributes.map(relation.output.indexOf(_)) + if (attributes.isEmpty) { + Seq(0) + } else { + attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + } val columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) val nextRow = new GenericMutableRow(columnAccessors.length) From c2048a5165b270f5baf2003fdfef7bc6c5875715 Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Tue, 15 Jul 2014 17:58:28 -0700 Subject: [PATCH 28/34] [SPARK-2498] [SQL] Synchronize on a lock when using scala reflection inside data type objects. JIRA ticket: https://issues.apache.org/jira/browse/SPARK-2498 Author: Zongheng Yang Closes #1423 from concretevitamin/scala-ref-catalyst and squashes the following commits: 325a149 [Zongheng Yang] Synchronize on a lock when initializing data type objects in Catalyst. --- .../spark/sql/catalyst/types/dataTypes.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index bb77bccf86176..cd4b5e9c1b529 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -19,17 +19,20 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp -import scala.util.parsing.combinator.RegexParsers - import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} +import scala.util.parsing.combinator.RegexParsers import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.util.Utils /** - * + * A JVM-global lock that should be used to prevent thread safety issues when using things in + * scala.reflect.*. Note that Scala Reflection API is made thread-safe in 2.11, but not yet for + * 2.10.* builds. See SI-6240 for more details. */ +protected[catalyst] object ScalaReflectionLock + object DataType extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "StringType" ^^^ StringType | @@ -62,7 +65,6 @@ object DataType extends RegexParsers { "true" ^^^ true | "false" ^^^ false - protected lazy val structType: Parser[DataType] = "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { case fields => new StructType(fields) @@ -106,7 +108,7 @@ abstract class NativeType extends DataType { @transient val tag: TypeTag[JvmType] val ordering: Ordering[JvmType] - @transient val classTag = { + @transient val classTag = ScalaReflectionLock.synchronized { val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) } @@ -114,22 +116,24 @@ abstract class NativeType extends DataType { case object StringType extends NativeType with PrimitiveType { type JvmType = String - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val ordering = implicitly[Ordering[JvmType]] } + case object BinaryType extends DataType with PrimitiveType { type JvmType = Array[Byte] } + case object BooleanType extends NativeType with PrimitiveType { type JvmType = Boolean - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val ordering = implicitly[Ordering[JvmType]] } case object TimestampType extends NativeType { type JvmType = Timestamp - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) @@ -159,7 +163,7 @@ abstract class IntegralType extends NumericType { case object LongType extends IntegralType { type JvmType = Long - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Long]] val integral = implicitly[Integral[Long]] val ordering = implicitly[Ordering[JvmType]] @@ -167,7 +171,7 @@ case object LongType extends IntegralType { case object IntegerType extends IntegralType { type JvmType = Int - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Int]] val integral = implicitly[Integral[Int]] val ordering = implicitly[Ordering[JvmType]] @@ -175,7 +179,7 @@ case object IntegerType extends IntegralType { case object ShortType extends IntegralType { type JvmType = Short - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Short]] val integral = implicitly[Integral[Short]] val ordering = implicitly[Ordering[JvmType]] @@ -183,7 +187,7 @@ case object ShortType extends IntegralType { case object ByteType extends IntegralType { type JvmType = Byte - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Byte]] val integral = implicitly[Integral[Byte]] val ordering = implicitly[Ordering[JvmType]] @@ -202,7 +206,7 @@ abstract class FractionalType extends NumericType { case object DecimalType extends FractionalType { type JvmType = BigDecimal - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[BigDecimal]] val fractional = implicitly[Fractional[BigDecimal]] val ordering = implicitly[Ordering[JvmType]] @@ -210,7 +214,7 @@ case object DecimalType extends FractionalType { case object DoubleType extends FractionalType { type JvmType = Double - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Double]] val fractional = implicitly[Fractional[Double]] val ordering = implicitly[Ordering[JvmType]] @@ -218,7 +222,7 @@ case object DoubleType extends FractionalType { case object FloatType extends FractionalType { type JvmType = Float - @transient lazy val tag = typeTag[JvmType] + @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } val numeric = implicitly[Numeric[Float]] val fractional = implicitly[Fractional[Float]] val ordering = implicitly[Ordering[JvmType]] From 4576d80a5155c9fbfebe9c36cca06c208bca5bd3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 15 Jul 2014 18:47:39 -0700 Subject: [PATCH 29/34] [SPARK-2469] Use Snappy (instead of LZF) for default shuffle compression codec This reduces shuffle compression memory usage by 3x. Author: Reynold Xin Closes #1415 from rxin/snappy and squashes the following commits: 06c1a01 [Reynold Xin] SPARK-2469: Use Snappy (instead of LZF) for default shuffle compression codec. --- .../src/main/scala/org/apache/spark/io/CompressionCodec.scala | 4 ++-- .../scala/org/apache/spark/io/CompressionCodecSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 33402c927c732..1b66218d86dd9 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -56,7 +56,7 @@ private[spark] object CompressionCodec { ctor.newInstance(conf).asInstanceOf[CompressionCodec] } - val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName + val DEFAULT_COMPRESSION_CODEC = classOf[SnappyCompressionCodec].getName } @@ -103,7 +103,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. - * Block size can be configured by spark.io.compression.snappy.block.size. + * Block size can be configured by `spark.io.compression.snappy.block.size`. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 42fc395fa698d..3f882a724b047 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -46,7 +46,7 @@ class CompressionCodecSuite extends FunSuite { test("default compression codec") { val codec = CompressionCodec.createCodec(conf) - assert(codec.getClass === classOf[LZFCompressionCodec]) + assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } From 9c12de5092312319aa22f24df47a6de0e41a0102 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 15 Jul 2014 21:21:52 -0700 Subject: [PATCH 30/34] [SPARK-2500] Move the logInfo for registering BlockManager to BlockManagerMasterActor.register method PR for SPARK-2500 Move the logInfo call for BlockManager to BlockManagerMasterActor.register instead of BlockManagerInfo constructor. Previously the loginfo call for registering the registering a BlockManager is happening in the BlockManagerInfo constructor. This kind of confusing because the code could call "new BlockManagerInfo" without actually registering a BlockManager and could confuse when reading the log files. Author: Henry Saputra Closes #1424 from hsaputra/move_registerblockmanager_log_to_registration_method and squashes the following commits: 3370b4a [Henry Saputra] Move the loginfo for BlockManager to BlockManagerMasterActor.register instead of BlockManagerInfo constructor. --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 6aed322eeb185..de1cc5539fb48 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -336,6 +336,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case None => blockManagerIdByExecutor(id.executorId) = id } + + logInfo("Registering block manager %s with %s RAM".format( + id.hostPort, Utils.bytesToString(maxMemSize))) + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } @@ -432,9 +436,6 @@ private[spark] class BlockManagerInfo( // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.bytesToString(maxMem))) - def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) def updateLastSeenMs() { From 563acf5edfbfb2fa756a1f0accde0940592663e9 Mon Sep 17 00:00:00 2001 From: Ken Takagiwa Date: Tue, 15 Jul 2014 21:34:05 -0700 Subject: [PATCH 31/34] follow pep8 None should be compared using is or is not http://legacy.python.org/dev/peps/pep-0008/ ## Programming Recommendations - Comparisons to singletons like None should always be done with is or is not, never the equality operators. Author: Ken Takagiwa Closes #1422 from giwa/apache_master and squashes the following commits: 7b361f3 [Ken Takagiwa] follow pep8 None should be checked using is or is not --- python/pyspark/cloudpickle.py | 4 ++-- python/pyspark/conf.py | 4 ++-- python/pyspark/rddsampler.py | 2 +- python/pyspark/shell.py | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index eb5dbb8de2b39..4fda2a9b950b8 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -243,10 +243,10 @@ def save_function(self, obj, name=None, pack=struct.pack): # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule == None: + if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: #Force server to import modules that have been imported in main modList = None - if themodule == None and not self.savedForceImports: + if themodule is None and not self.savedForceImports: mainmod = sys.modules['__main__'] if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): modList = list(mainmod.___pyc_forcedImports__) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 8eff4a242a529..60fc6ba7c52c2 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -30,7 +30,7 @@ u'local' >>> sc.appName u'My app' ->>> sc.sparkHome == None +>>> sc.sparkHome is None True >>> conf = SparkConf(loadDefaults=False) @@ -116,7 +116,7 @@ def setSparkHome(self, value): def setExecutorEnv(self, key=None, value=None, pairs=None): """Set an environment variable to be passed to executors.""" - if (key != None and pairs != None) or (key == None and pairs == None): + if (key is not None and pairs is not None) or (key is None and pairs is None): raise Exception("Either pass one key-value pair or a list of pairs") elif key != None: self._jconf.setExecutorEnv(key, value) diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 845a267e311c5..122bc38b03b0c 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -82,7 +82,7 @@ def getPoissonSample(self, split, mean): return (num_arrivals - 1) def shuffle(self, vals): - if self._random == None: + if self._random is None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index ebd714db7a918..2ce5409cd67c2 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,7 +35,7 @@ from pyspark.storagelevel import StorageLevel # this is the equivalent of ADD_JARS -add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None +add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") is not None else None if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) @@ -55,7 +55,7 @@ platform.python_build()[1])) print("SparkContext available as sc.") -if add_files != None: +if add_files is not None: print("Adding files: [%s]" % ", ".join(add_files)) # The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, From 90ca532a0fd95dc85cff8c5722d371e8368b2687 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Tue, 15 Jul 2014 21:35:36 -0700 Subject: [PATCH 32/34] [SPARK-2314][SQL] Override collect and take in JavaSchemaRDD, forwarding to SchemaRDD implementations. Author: Aaron Staple Closes #1421 from staple/SPARK-2314 and squashes the following commits: 73e04dc [Aaron Staple] [SPARK-2314] Override collect and take in JavaSchemaRDD, forwarding to SchemaRDD implementations. --- .../spark/sql/api/java/JavaSchemaRDD.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index aff6ffe9f3478..8fbf13b8b0150 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.api.java +import java.util.{List => JList} + import org.apache.spark.Partitioner import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} import org.apache.spark.api.java.function.{Function => JFunction} @@ -96,6 +98,20 @@ class JavaSchemaRDD( this } + // Overridden actions from JavaRDDLike. + + override def collect(): JList[Row] = { + import scala.collection.JavaConversions._ + val arr: java.util.Collection[Row] = baseSchemaRDD.collect().toSeq.map(new Row(_)) + new java.util.ArrayList(arr) + } + + override def take(num: Int): JList[Row] = { + import scala.collection.JavaConversions._ + val arr: java.util.Collection[Row] = baseSchemaRDD.take(num).toSeq.map(new Row(_)) + new java.util.ArrayList(arr) + } + // Transformations (return a new RDD) /** From 9b38b7c71352bb5e6d359515111ad9ca33299127 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 15 Jul 2014 22:35:34 -0700 Subject: [PATCH 33/34] [SPARK-2509][SQL] Add optimization for Substring. `Substring` including `null` literal cases could be added to `NullPropagation`. Author: Takuya UESHIN Closes #1428 from ueshin/issues/SPARK-2509 and squashes the following commits: d9eb85f [Takuya UESHIN] Add Substring cases to NullPropagation. --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 3 +++ 1 file changed, 3 insertions(+) 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 a142310c501b0..714e2cdac2b19 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 @@ -171,6 +171,9 @@ object NullPropagation extends Rule[LogicalPlan] { case Literal(candidate, _) if candidate == v => true case _ => false })) => Literal(true, BooleanType) + case e @ Substring(Literal(null, _), _, _) => Literal(null, e.dataType) + case e @ Substring(_, Literal(null, _), _) => Literal(null, e.dataType) + case e @ Substring(_, _, Literal(null, _)) => Literal(null, e.dataType) // Put exceptional cases above if any case e: BinaryArithmetic => e.children match { case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) From 632fb3d9a9ebb3d2218385403145d5b89c41c025 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 15 Jul 2014 22:43:48 -0700 Subject: [PATCH 34/34] [SPARK-2504][SQL] Fix nullability of Substring expression. This is a follow-up of #1359 with nullability narrowing. Author: Takuya UESHIN Closes #1426 from ueshin/issues/SPARK-2504 and squashes the following commits: 5157832 [Takuya UESHIN] Remove unnecessary white spaces. 80958ac [Takuya UESHIN] Fix nullability of Substring expression. --- .../expressions/stringOperations.scala | 24 +++++++++---------- .../ExpressionEvaluationSuite.scala | 14 +++++++---- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 4bd7bf5a0cd8c..f1b27c3cb517e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -215,19 +215,19 @@ case class EndsWith(left: Expression, right: Expression) case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression { type EvaluatedType = Any - - def nullable: Boolean = true + + def nullable: Boolean = str.nullable || pos.nullable || len.nullable def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved") } if (str.dataType == BinaryType) str.dataType else StringType } - + def references = children.flatMap(_.references).toSet - + override def children = str :: pos :: len :: Nil - + @inline def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { @@ -237,40 +237,40 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends // refers to element i-1 in the sequence. If a start index i is less than 0, it refers // to the -ith element before the end of the sequence. If a start index i is 0, it // refers to the first element. - + val start = startPos match { case pos if pos > 0 => pos - 1 case neg if neg < 0 => len + neg case _ => 0 } - + val end = sliceLen match { case max if max == Integer.MAX_VALUE => max case x => start + x } - + str.slice(start, end) } - + override def eval(input: Row): Any = { val string = str.eval(input) val po = pos.eval(input) val ln = len.eval(input) - + if ((string == null) || (po == null) || (ln == null)) { null } else { val start = po.asInstanceOf[Int] val length = ln.asInstanceOf[Int] - + string match { case ba: Array[Byte] => slice(ba, start, length) case other => slice(other.toString, start, length) } } } - + override def toString = len match { case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)" case _ => s"SUBSTR($str, $pos, $len)" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index f1d7aedcc2d2d..143330bd64716 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -469,9 +469,9 @@ class ExpressionEvaluationSuite extends FunSuite { test("Substring") { val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) - + val s = 'a.string.at(0) - + // substring from zero position with less-than-full length checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row) checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row) @@ -501,7 +501,7 @@ class ExpressionEvaluationSuite extends FunSuite { // substring(null, _, _) -> null checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null))) - + // substring(_, null, _) -> null checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row) @@ -514,6 +514,12 @@ class ExpressionEvaluationSuite extends FunSuite { // 2-arg substring from nonzero position checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row) + + val s_notNull = 'a.string.notNull.at(0) + + assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) + assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) } } -